mirror of https://github.com/apache/lucene.git
LUCENE-3490: restructure codec hierarchy
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1197603 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
033cfff007
commit
fa6500fa6c
|
@ -1,7 +1,9 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<classpath>
|
||||
<classpathentry kind="src" path="lucene/src/test-framework/java"/>
|
||||
<classpathentry kind="src" output="bin.tests-framework" path="lucene/src/test-framework/resources"/>
|
||||
<classpathentry kind="src" path="lucene/src/java"/>
|
||||
<classpathentry kind="src" path="lucene/src/test-framework"/>
|
||||
<classpathentry kind="src" path="lucene/src/resources"/>
|
||||
<classpathentry kind="src" path="lucene/src/test"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/demo/src/java"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/demo/src/resources"/>
|
||||
|
@ -13,6 +15,7 @@
|
|||
<classpathentry kind="src" path="lucene/contrib/memory/src/java"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/memory/src/test"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/misc/src/java"/>
|
||||
<classpathentry kind="src" output="bin.misc" path="lucene/contrib/misc/src/resources"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/misc/src/test"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/sandbox/src/java"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/sandbox/src/test"/>
|
||||
|
|
|
@ -36,6 +36,7 @@
|
|||
<entry name="?*.xsl" />
|
||||
<entry name="?*.vm" />
|
||||
<entry name="?*.zip" />
|
||||
<entry name="org.apache.lucene.index.codecs*" />
|
||||
<entry name="README*" />
|
||||
</wildcardResourcePatterns>
|
||||
</component>
|
||||
|
|
|
@ -5,6 +5,7 @@
|
|||
<output-test url="file://$MODULE_DIR$/../../build/contrib/misc/classes/test" />
|
||||
<exclude-output />
|
||||
<content url="file://$MODULE_DIR$">
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/resources" isTestSource="false" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
|
||||
</content>
|
||||
|
|
|
@ -5,9 +5,11 @@
|
|||
<output-test url="file://$MODULE_DIR$/build/classes/test" />
|
||||
<exclude-output />
|
||||
<content url="file://$MODULE_DIR$">
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/test-framework/resources" isTestSource="true" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/test-framework/java" isTestSource="true" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/resources" isTestSource="false" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/tools/java" isTestSource="false" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/test-framework" isTestSource="true" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
|
||||
<excludeFolder url="file://$MODULE_DIR$/build" />
|
||||
<excludeFolder url="file://$MODULE_DIR$/dist" />
|
||||
|
|
|
@ -219,7 +219,7 @@ Changes in Runtime Behavior
|
|||
* LUCENE-2881: FieldInfos is now tracked per segment. Before it was tracked
|
||||
per IndexWriter session, which resulted in FieldInfos that had the FieldInfo
|
||||
properties from all previous segments combined. Field numbers are now tracked
|
||||
globally across IndexWriter sessions and persisted into a X.fnx file on
|
||||
globally across IndexWriter sessions and persisted into a _X.fnx file on
|
||||
successful commit. The corresponding file format changes are backwards-
|
||||
compatible. (Michael Busch, Simon Willnauer)
|
||||
|
||||
|
@ -379,9 +379,8 @@ New features
|
|||
* LUCENE-1458, LUCENE-2111: With flexible indexing it is now possible
|
||||
for an application to create its own postings codec, to alter how
|
||||
fields, terms, docs and positions are encoded into the index. The
|
||||
standard codec is the default codec. Both IndexWriter and
|
||||
IndexReader accept a CodecProvider class to obtain codecs for newly
|
||||
written segments as well as existing segments opened for reading.
|
||||
standard codec is the default codec. IndexWriter accepts a Codec
|
||||
class to obtain codecs for newly written segments.
|
||||
|
||||
* LUCENE-1458, LUCENE-2111: Some experimental codecs have been added
|
||||
for flexible indexing, including pulsing codec (inlines
|
||||
|
@ -407,7 +406,7 @@ New features
|
|||
* LUCENE-2489: Added PerFieldCodecWrapper (in oal.index.codecs) which
|
||||
lets you set the Codec per field (Mike McCandless)
|
||||
|
||||
* LUCENE-2373: Extend CodecProvider to use SegmentInfosWriter and
|
||||
* LUCENE-2373: Extend Codec to use SegmentInfosWriter and
|
||||
SegmentInfosReader to allow customization of SegmentInfos data.
|
||||
(Andrzej Bialecki)
|
||||
|
||||
|
@ -449,10 +448,10 @@ New features
|
|||
(i.e. \* or "*") Custom QueryParser subclasses overriding getRangeQuery()
|
||||
will be passed null for any open endpoint. (Adriano Crestani, yonik)
|
||||
|
||||
* LUCENE-2742: Add native per-field codec support. CodecProvider lets you now
|
||||
register a codec for each field and which is in turn recorded in the segment
|
||||
and field information. Codecs are maintained on a per-segment basis and be
|
||||
resolved without knowing the actual codec used for writing the segment.
|
||||
* LUCENE-2742: Add native per-field postings format support. Codec lets you now
|
||||
register a postings format for each field and which is in turn recorded
|
||||
into the index. Postings formtas are maintained on a per-segment basis and be
|
||||
resolved without knowing the actual postings format used for writing the segment.
|
||||
(Simon Willnauer)
|
||||
|
||||
* LUCENE-2741: Add support for multiple codecs that use the same file
|
||||
|
|
|
@ -30,15 +30,15 @@
|
|||
</path>
|
||||
|
||||
<path id="test.classpath">
|
||||
<path refid="classpath"/>
|
||||
<path refid="junit-path"/>
|
||||
<pathelement location="${build.dir}/classes/test-framework"/>
|
||||
<path refid="classpath"/>
|
||||
<path refid="junit-path"/>
|
||||
<pathelement location="${build.dir}/classes/test"/>
|
||||
</path>
|
||||
|
||||
<path id="junit.classpath">
|
||||
<path refid="junit-path"/>
|
||||
<pathelement location="${build.dir}/classes/test-framework"/>
|
||||
<path refid="junit-path"/>
|
||||
<pathelement location="${build.dir}/classes/test"/>
|
||||
<pathelement location="${build.dir}/classes/java"/>
|
||||
<pathelement path="${java.class.path}"/>
|
||||
|
@ -556,11 +556,11 @@
|
|||
<sequential>
|
||||
<mkdir dir="${javadoc.dir}/test-framework"/>
|
||||
<invoke-javadoc
|
||||
overview="src/test-framework/overview.html"
|
||||
overview="src/test-framework/java/overview.html"
|
||||
destdir="${javadoc.dir}/test-framework"
|
||||
title="${Name} ${version} Test Framework API">
|
||||
<sources>
|
||||
<packageset dir="src/test-framework"/>
|
||||
<packageset dir="src/test-framework/java"/>
|
||||
<link href=""/>
|
||||
</sources>
|
||||
</invoke-javadoc>
|
||||
|
|
|
@ -83,8 +83,7 @@
|
|||
</or>
|
||||
</condition>
|
||||
<property name="tests.multiplier" value="1" />
|
||||
<property name="tests.codec" value="randomPerField" />
|
||||
<property name="tests.codecprovider" value="random" />
|
||||
<property name="tests.postingsformat" value="random" />
|
||||
<property name="tests.locale" value="random" />
|
||||
<property name="tests.timezone" value="random" />
|
||||
<property name="tests.directory" value="random" />
|
||||
|
@ -473,8 +472,13 @@
|
|||
</path>
|
||||
|
||||
<target name="compile-test-framework" depends="compile-core">
|
||||
<compile-test-macro srcdir="${tests-framework.src.dir}" destdir="${common.dir}/build/classes/test-framework"
|
||||
<compile-test-macro srcdir="${tests-framework.src.dir}/java" destdir="${common.dir}/build/classes/test-framework"
|
||||
test.classpath="test-framework.classpath"/>
|
||||
<!-- Copy the resources folder (if existent) -->
|
||||
<copy todir="${build.dir}/classes/test-framework" includeEmptyDirs="false">
|
||||
<globmapper from="resources/*" to="*" handledirsep="yes"/>
|
||||
<fileset dir="${tests-framework.src.dir}" includes="resources/**"/>
|
||||
</copy>
|
||||
</target>
|
||||
|
||||
<target name="compile-tools">
|
||||
|
@ -551,9 +555,7 @@
|
|||
<!-- directory for formatter lock -->
|
||||
<sysproperty key="tests.lockdir" value="${tests.lockdir}"/>
|
||||
<!-- set the codec tests should run with -->
|
||||
<sysproperty key="tests.codec" value="${tests.codec}"/>
|
||||
<!-- set the codec provider tests should run with -->
|
||||
<sysproperty key="tests.codecprovider" value="${tests.codecprovider}"/>
|
||||
<sysproperty key="tests.postingsformat" value="${tests.postingsformat}"/>
|
||||
<!-- set the locale tests should run with -->
|
||||
<sysproperty key="tests.locale" value="${tests.locale}"/>
|
||||
<!-- set the timezone tests should run with -->
|
||||
|
|
|
@ -39,8 +39,8 @@
|
|||
<path id="classpath" refid="base.classpath"/>
|
||||
|
||||
<path id="test.base.classpath">
|
||||
<path refid="classpath"/>
|
||||
<pathelement location="${common.dir}/build/classes/test-framework"/>
|
||||
<path refid="classpath"/>
|
||||
<path refid="junit-path"/>
|
||||
<pathelement location="${build.dir}/classes/java"/>
|
||||
</path>
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.lucene.document.Field;
|
|||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.queryparser.classic.QueryParser;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
|
@ -107,7 +108,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase {
|
|||
Directory ramdir = newDirectory();
|
||||
Analyzer analyzer = randomAnalyzer();
|
||||
IndexWriter writer = new IndexWriter(ramdir,
|
||||
new IndexWriterConfig(TEST_VERSION_CURRENT, analyzer).setCodecProvider(_TestUtil.alwaysCodec("Standard")));
|
||||
new IndexWriterConfig(TEST_VERSION_CURRENT, analyzer).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
|
||||
Document doc = new Document();
|
||||
Field field1 = newField("foo", fooField.toString(), TextField.TYPE_UNSTORED);
|
||||
Field field2 = newField("term", termField.toString(), TextField.TYPE_UNSTORED);
|
||||
|
|
|
@ -27,7 +27,6 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.IndexWriter; // Required for javadocs
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
|
||||
/**
|
||||
|
@ -54,8 +53,6 @@ import org.apache.lucene.store.FSDirectory;
|
|||
*/
|
||||
public class IndexSplitter {
|
||||
public SegmentInfos infos;
|
||||
|
||||
private final CodecProvider codecs;
|
||||
|
||||
FSDirectory fsDir;
|
||||
|
||||
|
@ -96,17 +93,12 @@ public class IndexSplitter {
|
|||
is.split(targetDir, segs.toArray(new String[0]));
|
||||
}
|
||||
}
|
||||
|
||||
public IndexSplitter(File dir) throws IOException {
|
||||
this(dir, CodecProvider.getDefault());
|
||||
}
|
||||
|
||||
public IndexSplitter(File dir, CodecProvider codecs) throws IOException {
|
||||
public IndexSplitter(File dir) throws IOException {
|
||||
this.dir = dir;
|
||||
this.codecs = codecs;
|
||||
fsDir = FSDirectory.open(dir);
|
||||
infos = new SegmentInfos(codecs);
|
||||
infos.read(fsDir, codecs);
|
||||
infos = new SegmentInfos();
|
||||
infos.read(fsDir);
|
||||
}
|
||||
|
||||
public void listSegments() throws IOException {
|
||||
|
@ -140,13 +132,13 @@ public class IndexSplitter {
|
|||
infos.remove(idx);
|
||||
}
|
||||
infos.changed();
|
||||
infos.commit(fsDir);
|
||||
infos.commit(fsDir, infos.codecFormat());
|
||||
}
|
||||
|
||||
public void split(File destDir, String[] segs) throws IOException {
|
||||
destDir.mkdirs();
|
||||
FSDirectory destFSDir = FSDirectory.open(destDir);
|
||||
SegmentInfos destInfos = new SegmentInfos(codecs);
|
||||
SegmentInfos destInfos = new SegmentInfos();
|
||||
destInfos.counter = infos.counter;
|
||||
for (String n : segs) {
|
||||
SegmentInfo info = getInfo(n);
|
||||
|
@ -160,7 +152,7 @@ public class IndexSplitter {
|
|||
}
|
||||
}
|
||||
destInfos.changed();
|
||||
destInfos.commit(destFSDir);
|
||||
destInfos.commit(destFSDir, infos.codecFormat());
|
||||
// System.out.println("destDir:"+destDir.getAbsolutePath());
|
||||
}
|
||||
|
||||
|
|
|
@ -17,33 +17,17 @@ package org.apache.lucene.index.codecs.appending;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.BlockTermsReader;
|
||||
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
|
||||
|
||||
/**
|
||||
* This codec extends {@link StandardCodec} to work on append-only outputs, such
|
||||
* This codec extends {@link Lucene40Codec} to work on append-only outputs, such
|
||||
* as plain output streams and append-only filesystems.
|
||||
*
|
||||
* <p>Note: compound file format feature is not compatible with
|
||||
|
@ -54,105 +38,32 @@ import org.apache.lucene.util.BytesRef;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public class AppendingCodec extends Codec {
|
||||
public static String CODEC_NAME = "Appending";
|
||||
|
||||
public AppendingCodec() {
|
||||
super(CODEC_NAME);
|
||||
super("Appending");
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
|
||||
throws IOException {
|
||||
PostingsWriterBase docsWriter = new StandardPostingsWriter(state);
|
||||
boolean success = false;
|
||||
AppendingTermsIndexWriter indexWriter = null;
|
||||
try {
|
||||
indexWriter = new AppendingTermsIndexWriter(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docsWriter.close();
|
||||
}
|
||||
}
|
||||
success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new AppendingTermsDictWriter(indexWriter, state, docsWriter);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
docsWriter.close();
|
||||
} finally {
|
||||
indexWriter.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state)
|
||||
throws IOException {
|
||||
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
|
||||
TermsIndexReaderBase indexReader;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new AppendingTermsIndexReader(state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
state.termsIndexDivisor,
|
||||
BytesRef.getUTF8SortedAsUnicodeComparator(),
|
||||
state.codecId, state.context);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docsReader.close();
|
||||
}
|
||||
}
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new AppendingTermsDictReader(indexReader,
|
||||
state.dir, state.fieldInfos, state.segmentInfo.name,
|
||||
docsReader,
|
||||
state.context,
|
||||
StandardCodec.TERMS_CACHE_SIZE,
|
||||
state.codecId);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
docsReader.close();
|
||||
} finally {
|
||||
indexReader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files)
|
||||
throws IOException {
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
StandardCodec.getStandardExtensions(extensions);
|
||||
DefaultDocValuesConsumer.getExtensions(extensions);
|
||||
}
|
||||
private final PostingsFormat postings = new AppendingPostingsFormat();
|
||||
private final SegmentInfosFormat infos = new AppendingSegmentInfosFormat();
|
||||
private final FieldsFormat fields = new DefaultFieldsFormat();
|
||||
private final DocValuesFormat docValues = new DefaultDocValuesFormat();
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new DefaultDocValuesConsumer(state);
|
||||
public PostingsFormat postingsFormat() {
|
||||
return postings;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new DefaultDocValuesProducer(state);
|
||||
public FieldsFormat fieldsFormat() {
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat docValuesFormat() {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentInfosFormat segmentInfosFormat() {
|
||||
return infos;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,128 @@
|
|||
package org.apache.lucene.index.codecs.appending;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsReader;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsWriter;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.codecs.BlockTermsReader;
|
||||
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Appending postings impl
|
||||
*/
|
||||
class AppendingPostingsFormat extends PostingsFormat {
|
||||
public static String CODEC_NAME = "Appending";
|
||||
|
||||
public AppendingPostingsFormat() {
|
||||
super(CODEC_NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
|
||||
throws IOException {
|
||||
PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
|
||||
boolean success = false;
|
||||
AppendingTermsIndexWriter indexWriter = null;
|
||||
try {
|
||||
indexWriter = new AppendingTermsIndexWriter(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docsWriter.close();
|
||||
}
|
||||
}
|
||||
success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new AppendingTermsDictWriter(indexWriter, state, docsWriter);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
docsWriter.close();
|
||||
} finally {
|
||||
indexWriter.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state)
|
||||
throws IOException {
|
||||
PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
TermsIndexReaderBase indexReader;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new AppendingTermsIndexReader(state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
state.termsIndexDivisor,
|
||||
BytesRef.getUTF8SortedAsUnicodeComparator(),
|
||||
state.segmentSuffix, state.context);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docsReader.close();
|
||||
}
|
||||
}
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new AppendingTermsDictReader(indexReader,
|
||||
state.dir, state.fieldInfos, state.segmentInfo.name,
|
||||
docsReader,
|
||||
state.context,
|
||||
Lucene40PostingsFormat.TERMS_CACHE_SIZE,
|
||||
state.segmentSuffix);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
docsReader.close();
|
||||
} finally {
|
||||
indexReader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files)
|
||||
throws IOException {
|
||||
Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
package org.apache.lucene.index.codecs.appending;
|
||||
|
||||
/**
|
||||
* 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.index.codecs.DefaultSegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosWriter;
|
||||
|
||||
public class AppendingSegmentInfosFormat extends DefaultSegmentInfosFormat {
|
||||
private final SegmentInfosWriter writer = new AppendingSegmentInfosWriter();
|
||||
|
||||
@Override
|
||||
public SegmentInfosWriter getSegmentInfosWriter() {
|
||||
return writer;
|
||||
}
|
||||
}
|
|
@ -20,8 +20,6 @@ package org.apache.lucene.index.codecs.appending;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
||||
public class AppendingSegmentInfosWriter extends DefaultSegmentInfosWriter {
|
||||
|
|
|
@ -34,9 +34,9 @@ public class AppendingTermsDictReader extends BlockTermsReader {
|
|||
public AppendingTermsDictReader(TermsIndexReaderBase indexReader,
|
||||
Directory dir, FieldInfos fieldInfos, String segment,
|
||||
PostingsReaderBase postingsReader, IOContext context,
|
||||
int termsCacheSize, int codecId) throws IOException {
|
||||
int termsCacheSize, String segmentSuffix) throws IOException {
|
||||
super(indexReader, dir, fieldInfos, segment, postingsReader, context,
|
||||
termsCacheSize, codecId);
|
||||
termsCacheSize, segmentSuffix);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -31,9 +31,9 @@ import org.apache.lucene.util.CodecUtil;
|
|||
public class AppendingTermsIndexReader extends FixedGapTermsIndexReader {
|
||||
|
||||
public AppendingTermsIndexReader(Directory dir, FieldInfos fieldInfos,
|
||||
String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId, IOContext context)
|
||||
String segment, int indexDivisor, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
|
||||
throws IOException {
|
||||
super(dir, fieldInfos, segment, indexDivisor, termComp, codecId, context);
|
||||
super(dir, fieldInfos, segment, indexDivisor, termComp, segmentSuffix, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,16 @@
|
|||
# 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.
|
||||
|
||||
org.apache.lucene.index.codecs.appending.AppendingCodec
|
|
@ -27,7 +27,9 @@ import org.apache.lucene.search.SearcherWarmer;
|
|||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.NRTCachingDirectory;
|
||||
import org.apache.lucene.util.LuceneTestCase.UseNoMemoryExpensiveCodec;
|
||||
|
||||
@UseNoMemoryExpensiveCodec
|
||||
public class TestNRTManager extends ThreadedIndexingAndSearchingTestCase {
|
||||
|
||||
private final ThreadLocal<Long> lastGens = new ThreadLocal<Long>();
|
||||
|
|
|
@ -34,11 +34,6 @@ import org.apache.lucene.index.MultiFields;
|
|||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosReader;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosReader;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
@ -50,30 +45,7 @@ import org.apache.lucene.util.Version;
|
|||
|
||||
public class TestAppendingCodec extends LuceneTestCase {
|
||||
|
||||
static class AppendingCodecProvider extends CodecProvider {
|
||||
Codec appending = new AppendingCodec();
|
||||
SegmentInfosWriter infosWriter = new AppendingSegmentInfosWriter();
|
||||
SegmentInfosReader infosReader = new DefaultSegmentInfosReader();
|
||||
public AppendingCodecProvider() {
|
||||
setDefaultFieldCodec(appending.name);
|
||||
}
|
||||
@Override
|
||||
public Codec lookup(String name) {
|
||||
return appending;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentInfosReader getSegmentInfosReader() {
|
||||
return infosReader;
|
||||
}
|
||||
@Override
|
||||
public SegmentInfosWriter getSegmentInfosWriter() {
|
||||
return infosWriter;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class AppendingIndexOutputWrapper extends IndexOutput {
|
||||
private static class AppendingIndexOutputWrapper extends IndexOutput {
|
||||
IndexOutput wrapped;
|
||||
|
||||
public AppendingIndexOutputWrapper(IndexOutput wrapped) {
|
||||
|
@ -137,7 +109,7 @@ public class TestAppendingCodec extends LuceneTestCase {
|
|||
Directory dir = new AppendingRAMDirectory(random, new RAMDirectory());
|
||||
IndexWriterConfig cfg = new IndexWriterConfig(Version.LUCENE_40, new MockAnalyzer(random));
|
||||
|
||||
cfg.setCodecProvider(new AppendingCodecProvider());
|
||||
cfg.setCodec(new AppendingCodec());
|
||||
((TieredMergePolicy)cfg.getMergePolicy()).setUseCompoundFile(false);
|
||||
IndexWriter writer = new IndexWriter(dir, cfg);
|
||||
Document doc = new Document();
|
||||
|
@ -151,7 +123,7 @@ public class TestAppendingCodec extends LuceneTestCase {
|
|||
writer.addDocument(doc);
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
IndexReader reader = IndexReader.open(dir, null, true, 1, new AppendingCodecProvider());
|
||||
IndexReader reader = IndexReader.open(dir, null, true, 1);
|
||||
assertEquals(2, reader.numDocs());
|
||||
Document doc2 = reader.document(0);
|
||||
assertEquals(text, doc2.get("f"));
|
||||
|
|
|
@ -34,9 +34,11 @@ import org.apache.lucene.index.Term;
|
|||
import org.apache.lucene.index.ThreadedIndexingAndSearchingTestCase;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase.UseNoMemoryExpensiveCodec;
|
||||
import org.apache.lucene.util.NamedThreadFactory;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
@UseNoMemoryExpensiveCodec
|
||||
public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
|
||||
|
||||
boolean warmCalled;
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -143,8 +143,8 @@ public class CheckIndex {
|
|||
/** Name of the segment. */
|
||||
public String name;
|
||||
|
||||
/** CodecInfo used to read this segment. */
|
||||
public SegmentCodecs codec;
|
||||
/** Codec used to read this segment. */
|
||||
public Codec codec;
|
||||
|
||||
/** Document count (does not take deletions into account). */
|
||||
public int docCount;
|
||||
|
@ -322,10 +322,6 @@ public class CheckIndex {
|
|||
public Status checkIndex() throws IOException {
|
||||
return checkIndex(null);
|
||||
}
|
||||
|
||||
public Status checkIndex(List<String> onlySegments) throws IOException {
|
||||
return checkIndex(onlySegments, CodecProvider.getDefault());
|
||||
}
|
||||
|
||||
/** Returns a {@link Status} instance detailing
|
||||
* the state of the index.
|
||||
|
@ -339,13 +335,13 @@ public class CheckIndex {
|
|||
* <p><b>WARNING</b>: make sure
|
||||
* you only call this when the index is not opened by any
|
||||
* writer. */
|
||||
public Status checkIndex(List<String> onlySegments, CodecProvider codecs) throws IOException {
|
||||
public Status checkIndex(List<String> onlySegments) throws IOException {
|
||||
NumberFormat nf = NumberFormat.getInstance();
|
||||
SegmentInfos sis = new SegmentInfos(codecs);
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
Status result = new Status();
|
||||
result.dir = dir;
|
||||
try {
|
||||
sis.read(dir, codecs);
|
||||
sis.read(dir);
|
||||
} catch (Throwable t) {
|
||||
msg("ERROR: could not read any segments file in directory");
|
||||
result.missingSegments = true;
|
||||
|
@ -377,6 +373,7 @@ public class CheckIndex {
|
|||
|
||||
final int numSegments = sis.size();
|
||||
final String segmentsFileName = sis.getCurrentSegmentFileName();
|
||||
// note: we only read the format byte (required preamble) here!
|
||||
IndexInput input = null;
|
||||
try {
|
||||
input = dir.openInput(segmentsFileName, IOContext.DEFAULT);
|
||||
|
@ -489,7 +486,7 @@ public class CheckIndex {
|
|||
SegmentReader reader = null;
|
||||
|
||||
try {
|
||||
final SegmentCodecs codec = info.getSegmentCodecs();
|
||||
final Codec codec = info.getCodec();
|
||||
msg(" codec=" + codec);
|
||||
segInfoStat.codec = codec;
|
||||
msg(" compound=" + info.getUseCompoundFile());
|
||||
|
@ -1182,11 +1179,11 @@ public class CheckIndex {
|
|||
*
|
||||
* <p><b>WARNING</b>: Make sure you only call this when the
|
||||
* index is not opened by any writer. */
|
||||
public void fixIndex(Status result) throws IOException {
|
||||
public void fixIndex(Status result, Codec codec) throws IOException {
|
||||
if (result.partial)
|
||||
throw new IllegalArgumentException("can only fix an index that was fully checked (this status checked a subset of segments)");
|
||||
result.newSegments.changed();
|
||||
result.newSegments.commit(result.dir);
|
||||
result.newSegments.commit(result.dir, codec);
|
||||
}
|
||||
|
||||
private static boolean assertsOn;
|
||||
|
@ -1236,6 +1233,7 @@ public class CheckIndex {
|
|||
public static void main(String[] args) throws IOException, InterruptedException {
|
||||
|
||||
boolean doFix = false;
|
||||
Codec codec = Codec.getDefault(); // only used when fixing
|
||||
boolean verbose = false;
|
||||
List<String> onlySegments = new ArrayList<String>();
|
||||
String indexPath = null;
|
||||
|
@ -1244,6 +1242,13 @@ public class CheckIndex {
|
|||
if (args[i].equals("-fix")) {
|
||||
doFix = true;
|
||||
i++;
|
||||
} else if (args[i].equals("-codec")) {
|
||||
if (i == args.length-1) {
|
||||
System.out.println("ERROR: missing name for -codec option");
|
||||
System.exit(1);
|
||||
}
|
||||
codec = Codec.forName(args[i+1]);
|
||||
i+=2;
|
||||
} else if (args[i].equals("-verbose")) {
|
||||
verbose = true;
|
||||
i++;
|
||||
|
@ -1269,6 +1274,7 @@ public class CheckIndex {
|
|||
System.out.println("\nUsage: java org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-segment X] [-segment Y]\n" +
|
||||
"\n" +
|
||||
" -fix: actually write a new segments_N file, removing any problematic segments\n" +
|
||||
" -codec X: when fixing, codec to write the new segments_N file with\n" +
|
||||
" -verbose: print additional details\n" +
|
||||
" -segment X: only check the specified segments. This can be specified multiple\n" +
|
||||
" times, to check more than one segment, eg '-segment _2 -segment _a'.\n" +
|
||||
|
@ -1329,7 +1335,7 @@ public class CheckIndex {
|
|||
System.out.println(" " + (5-s) + "...");
|
||||
}
|
||||
System.out.println("Writing...");
|
||||
checker.fixIndex(result);
|
||||
checker.fixIndex(result, codec);
|
||||
System.out.println("OK");
|
||||
System.out.println("Wrote new segments file \"" + result.newSegments.getCurrentSegmentFileName() + "\"");
|
||||
}
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.Lock;
|
||||
import org.apache.lucene.store.LockObtainFailedException;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -45,8 +44,6 @@ import org.apache.lucene.util.MapBackedSet;
|
|||
class DirectoryReader extends IndexReader implements Cloneable {
|
||||
protected Directory directory;
|
||||
protected boolean readOnly;
|
||||
|
||||
protected CodecProvider codecs;
|
||||
|
||||
IndexWriter writer;
|
||||
|
||||
|
@ -78,15 +75,13 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
// }
|
||||
|
||||
static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly,
|
||||
final int termInfosIndexDivisor, CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
final CodecProvider codecProvider = codecs == null ? CodecProvider.getDefault()
|
||||
: codecs;
|
||||
final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
|
||||
return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
|
||||
@Override
|
||||
protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
|
||||
SegmentInfos infos = new SegmentInfos(codecProvider);
|
||||
infos.read(directory, segmentFileName, codecProvider);
|
||||
return new DirectoryReader(directory, infos, deletionPolicy, readOnly, termInfosIndexDivisor, codecProvider);
|
||||
SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(directory, segmentFileName);
|
||||
return new DirectoryReader(directory, infos, deletionPolicy, readOnly, termInfosIndexDivisor);
|
||||
}
|
||||
}.run(commit);
|
||||
}
|
||||
|
@ -97,17 +92,12 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
// }
|
||||
|
||||
/** Construct reading the named set of readers. */
|
||||
DirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor, CodecProvider codecs) throws IOException {
|
||||
DirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor) throws IOException {
|
||||
this.directory = directory;
|
||||
this.readOnly = readOnly;
|
||||
this.segmentInfos = sis;
|
||||
this.deletionPolicy = deletionPolicy;
|
||||
this.termInfosIndexDivisor = termInfosIndexDivisor;
|
||||
if (codecs == null) {
|
||||
this.codecs = CodecProvider.getDefault();
|
||||
} else {
|
||||
this.codecs = codecs;
|
||||
}
|
||||
readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
applyAllDeletes = false;
|
||||
|
||||
|
@ -141,17 +131,12 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
}
|
||||
|
||||
// Used by near real-time search
|
||||
DirectoryReader(IndexWriter writer, SegmentInfos infos, CodecProvider codecs, boolean applyAllDeletes) throws IOException {
|
||||
DirectoryReader(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes) throws IOException {
|
||||
this.directory = writer.getDirectory();
|
||||
this.readOnly = true;
|
||||
this.applyAllDeletes = applyAllDeletes; // saved for reopen
|
||||
|
||||
this.termInfosIndexDivisor = writer.getConfig().getReaderTermsIndexDivisor();
|
||||
if (codecs == null) {
|
||||
this.codecs = CodecProvider.getDefault();
|
||||
} else {
|
||||
this.codecs = codecs;
|
||||
}
|
||||
readerFinishedListeners = writer.getReaderFinishedListeners();
|
||||
|
||||
// IndexWriter synchronizes externally before calling
|
||||
|
@ -200,8 +185,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
|
||||
/** This constructor is only used for {@link #doOpenIfChanged()} */
|
||||
DirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders,
|
||||
boolean readOnly, boolean doClone, int termInfosIndexDivisor, CodecProvider codecs,
|
||||
Collection<ReaderFinishedListener> readerFinishedListeners) throws IOException {
|
||||
boolean readOnly, boolean doClone, int termInfosIndexDivisor, Collection<ReaderFinishedListener> readerFinishedListeners) throws IOException {
|
||||
this.directory = directory;
|
||||
this.readOnly = readOnly;
|
||||
this.segmentInfos = infos;
|
||||
|
@ -209,13 +193,6 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
this.readerFinishedListeners = readerFinishedListeners;
|
||||
applyAllDeletes = false;
|
||||
|
||||
if (codecs == null) {
|
||||
this.codecs = CodecProvider.getDefault();
|
||||
} else {
|
||||
this.codecs = codecs;
|
||||
}
|
||||
|
||||
|
||||
// we put the old SegmentReaders in a map, that allows us
|
||||
// to lookup a reader using its segment name
|
||||
Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
|
||||
|
@ -347,7 +324,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
starts[subReaders.length] = maxDoc;
|
||||
|
||||
if (!readOnly) {
|
||||
maxIndexVersion = SegmentInfos.readCurrentVersion(directory, codecs);
|
||||
maxIndexVersion = SegmentInfos.readCurrentVersion(directory);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -498,15 +475,15 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
|
||||
@Override
|
||||
protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
|
||||
final SegmentInfos infos = new SegmentInfos(codecs);
|
||||
infos.read(directory, segmentFileName, codecs);
|
||||
final SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(directory, segmentFileName);
|
||||
return doOpenIfChanged(infos, false, openReadOnly);
|
||||
}
|
||||
}.run(commit);
|
||||
}
|
||||
|
||||
private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
|
||||
return new DirectoryReader(directory, infos, subReaders, openReadOnly, doClone, termInfosIndexDivisor, codecs, readerFinishedListeners);
|
||||
return new DirectoryReader(directory, infos, subReaders, openReadOnly, doClone, termInfosIndexDivisor, readerFinishedListeners);
|
||||
}
|
||||
|
||||
/** Version number when this IndexReader was opened. */
|
||||
|
@ -712,7 +689,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
|
||||
// we have to check whether index has changed since this reader was opened.
|
||||
// if so, this reader is no longer valid for deletion
|
||||
if (SegmentInfos.readCurrentVersion(directory, codecs) > maxIndexVersion) {
|
||||
if (SegmentInfos.readCurrentVersion(directory) > maxIndexVersion) {
|
||||
stale = true;
|
||||
this.writeLock.release();
|
||||
this.writeLock = null;
|
||||
|
@ -743,7 +720,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
// KeepOnlyLastCommitDeleter:
|
||||
IndexFileDeleter deleter = new IndexFileDeleter(directory,
|
||||
deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
|
||||
segmentInfos, null, codecs, null);
|
||||
segmentInfos, null, null);
|
||||
segmentInfos.updateGeneration(deleter.getLastSegmentInfos());
|
||||
segmentInfos.changed();
|
||||
|
||||
|
@ -764,7 +741,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
|
||||
// Sync all files we just wrote
|
||||
directory.sync(segmentInfos.files(directory, false));
|
||||
segmentInfos.commit(directory);
|
||||
segmentInfos.commit(directory, segmentInfos.codecFormat());
|
||||
success = true;
|
||||
} finally {
|
||||
|
||||
|
@ -842,7 +819,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
ensureOpen();
|
||||
if (writer == null || writer.isClosed()) {
|
||||
// we loaded SegmentInfos from the directory
|
||||
return SegmentInfos.readCurrentVersion(directory, codecs) == segmentInfos.getVersion();
|
||||
return SegmentInfos.readCurrentVersion(directory) == segmentInfos.getVersion();
|
||||
} else {
|
||||
return writer.nrtIsCurrent(segmentInfos);
|
||||
}
|
||||
|
@ -925,17 +902,12 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
|
||||
/** @see org.apache.lucene.index.IndexReader#listCommits */
|
||||
public static List<IndexCommit> listCommits(Directory dir) throws IOException {
|
||||
return listCommits(dir, CodecProvider.getDefault());
|
||||
}
|
||||
|
||||
/** @see org.apache.lucene.index.IndexReader#listCommits */
|
||||
public static List<IndexCommit> listCommits(Directory dir, CodecProvider codecs) throws IOException {
|
||||
final String[] files = dir.listAll();
|
||||
|
||||
List<IndexCommit> commits = new ArrayList<IndexCommit>();
|
||||
|
||||
SegmentInfos latest = new SegmentInfos(codecs);
|
||||
latest.read(dir, codecs);
|
||||
SegmentInfos latest = new SegmentInfos();
|
||||
latest.read(dir);
|
||||
final long currentGen = latest.getGeneration();
|
||||
|
||||
commits.add(new ReaderCommit(latest, dir));
|
||||
|
@ -948,11 +920,11 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
!fileName.equals(IndexFileNames.SEGMENTS_GEN) &&
|
||||
SegmentInfos.generationFromSegmentsFileName(fileName) < currentGen) {
|
||||
|
||||
SegmentInfos sis = new SegmentInfos(codecs);
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
try {
|
||||
// IOException allowed to throw there, in case
|
||||
// segments_N is corrupt
|
||||
sis.read(dir, fileName, codecs);
|
||||
sis.read(dir, fileName);
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
// LUCENE-948: on NFS (and maybe others), if
|
||||
// you have writers switching back and forth
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.HashSet;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
|
@ -320,14 +320,13 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
docValuesConsumerAndDocID.docID = docState.docID;
|
||||
return docValuesConsumerAndDocID.docValuesConsumer;
|
||||
}
|
||||
PerDocConsumer perDocConsumer = perDocConsumers.get(fieldInfo.getCodecId());
|
||||
|
||||
PerDocConsumer perDocConsumer = perDocConsumers.get(0);
|
||||
if (perDocConsumer == null) {
|
||||
PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState(fieldInfo.getCodecId());
|
||||
SegmentCodecs codecs = perDocWriteState.segmentCodecs;
|
||||
assert codecs.codecs.length > fieldInfo.getCodecId();
|
||||
Codec codec = codecs.codecs[fieldInfo.getCodecId()];
|
||||
perDocConsumer = codec.docsConsumer(perDocWriteState);
|
||||
perDocConsumers.put(Integer.valueOf(fieldInfo.getCodecId()), perDocConsumer);
|
||||
PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
|
||||
DocValuesFormat dvFormat = docState.docWriter.codec.docValuesFormat();
|
||||
perDocConsumer = dvFormat.docsConsumer(perDocWriteState);
|
||||
perDocConsumers.put(0, perDocConsumer);
|
||||
}
|
||||
boolean success = false;
|
||||
DocValuesConsumer docValuesConsumer = null;
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
|
|||
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
|
||||
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
||||
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
|
@ -125,8 +126,11 @@ final class DocumentsWriter {
|
|||
final DocumentsWriterPerThreadPool perThreadPool;
|
||||
final FlushPolicy flushPolicy;
|
||||
final DocumentsWriterFlushControl flushControl;
|
||||
DocumentsWriter(IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumberBiMap globalFieldNumbers,
|
||||
|
||||
final Codec codec;
|
||||
DocumentsWriter(Codec codec, IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumberBiMap globalFieldNumbers,
|
||||
BufferedDeletesStream bufferedDeletesStream) throws IOException {
|
||||
this.codec = codec;
|
||||
this.directory = directory;
|
||||
this.indexWriter = writer;
|
||||
this.similarityProvider = config.getSimilarityProvider();
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.text.NumberFormat;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
|
@ -152,7 +152,7 @@ public class DocumentsWriterPerThread {
|
|||
}
|
||||
private final static boolean INFO_VERBOSE = false;
|
||||
final DocumentsWriter parent;
|
||||
final CodecProvider codecProvider;
|
||||
final Codec codec;
|
||||
final IndexWriter writer;
|
||||
final Directory directory;
|
||||
final DocState docState;
|
||||
|
@ -183,7 +183,7 @@ public class DocumentsWriterPerThread {
|
|||
this.fieldInfos = fieldInfos;
|
||||
this.writer = parent.indexWriter;
|
||||
this.infoStream = parent.infoStream;
|
||||
this.codecProvider = this.writer.codecs;
|
||||
this.codec = parent.codec;
|
||||
this.docState = new DocState(this);
|
||||
this.docState.similarityProvider = parent.indexWriter.getConfig()
|
||||
.getSimilarityProvider();
|
||||
|
@ -405,8 +405,8 @@ public class DocumentsWriterPerThread {
|
|||
return numDocsInRAM;
|
||||
}
|
||||
|
||||
SegmentCodecs getCodec() {
|
||||
return flushState.segmentCodecs;
|
||||
Codec getCodec() {
|
||||
return flushState.codec;
|
||||
}
|
||||
|
||||
/** Reset after a flush */
|
||||
|
@ -443,7 +443,7 @@ public class DocumentsWriterPerThread {
|
|||
assert deleteSlice == null : "all deletes must be applied in prepareFlush";
|
||||
flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos,
|
||||
numDocsInRAM, writer.getConfig().getTermIndexInterval(),
|
||||
fieldInfos.buildSegmentCodecs(true), pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
|
||||
codec, pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
|
||||
final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
|
||||
// Apply delete-by-docID now (delete-byDocID only
|
||||
// happens when an exception is hit processing that
|
||||
|
@ -474,12 +474,12 @@ public class DocumentsWriterPerThread {
|
|||
try {
|
||||
consumer.flush(flushState);
|
||||
pendingDeletes.terms.clear();
|
||||
final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.segmentCodecs, fieldInfos.asReadOnly());
|
||||
final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly());
|
||||
if (infoStream != null) {
|
||||
message("new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.liveDocs.count())) + " deleted docs");
|
||||
message("new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
|
||||
message("flushedFiles=" + newSegment.files());
|
||||
message("flushed codecs=" + newSegment.getSegmentCodecs());
|
||||
message("flushed codec=" + newSegment.getCodec());
|
||||
}
|
||||
flushedDocCount += flushState.numDocs;
|
||||
|
||||
|
@ -556,9 +556,9 @@ public class DocumentsWriterPerThread {
|
|||
bytesUsed.addAndGet(-(length *(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT)));
|
||||
}
|
||||
|
||||
PerDocWriteState newPerDocWriteState(int codecId) {
|
||||
PerDocWriteState newPerDocWriteState(String segmentSuffix) {
|
||||
assert segment != null;
|
||||
return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, codecId, IOContext.DEFAULT);
|
||||
return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, segmentSuffix, IOContext.DEFAULT);
|
||||
}
|
||||
|
||||
void setInfoStream(PrintStream infoStream) {
|
||||
|
|
|
@ -20,8 +20,6 @@ import java.util.Iterator;
|
|||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
|
||||
import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.util.SetOnce;
|
||||
|
||||
/**
|
||||
|
@ -128,7 +126,6 @@ public abstract class DocumentsWriterPerThreadPool {
|
|||
|
||||
private final ThreadState[] perThreads;
|
||||
private volatile int numThreadStatesActive;
|
||||
private CodecProvider codecProvider;
|
||||
private FieldNumberBiMap globalFieldMap;
|
||||
private final SetOnce<DocumentsWriter> documentsWriter = new SetOnce<DocumentsWriter>();
|
||||
|
||||
|
@ -148,11 +145,9 @@ public abstract class DocumentsWriterPerThreadPool {
|
|||
|
||||
public void initialize(DocumentsWriter documentsWriter, FieldNumberBiMap globalFieldMap, IndexWriterConfig config) {
|
||||
this.documentsWriter.set(documentsWriter); // thread pool is bound to DW
|
||||
final CodecProvider codecs = config.getCodecProvider();
|
||||
this.codecProvider = codecs;
|
||||
this.globalFieldMap = globalFieldMap;
|
||||
for (int i = 0; i < perThreads.length; i++) {
|
||||
final FieldInfos infos = globalFieldMap.newFieldInfos(SegmentCodecsBuilder.create(codecs));
|
||||
final FieldInfos infos = new FieldInfos(globalFieldMap);
|
||||
perThreads[i] = new ThreadState(new DocumentsWriterPerThread(documentsWriter.directory, documentsWriter, infos, documentsWriter.chain));
|
||||
}
|
||||
}
|
||||
|
@ -240,7 +235,7 @@ public abstract class DocumentsWriterPerThreadPool {
|
|||
assert threadState.isHeldByCurrentThread();
|
||||
final DocumentsWriterPerThread dwpt = threadState.perThread;
|
||||
if (!closed) {
|
||||
final FieldInfos infos = globalFieldMap.newFieldInfos(SegmentCodecsBuilder.create(codecProvider));
|
||||
final FieldInfos infos = new FieldInfos(globalFieldMap);
|
||||
final DocumentsWriterPerThread newDwpt = new DocumentsWriterPerThread(dwpt, infos);
|
||||
newDwpt.initialize();
|
||||
threadState.resetWriter(newDwpt);
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.lucene.index.values.ValueType;
|
|||
|
||||
/** @lucene.experimental */
|
||||
public final class FieldInfo {
|
||||
public static final int UNASSIGNED_CODEC_ID = -1;
|
||||
public final String name;
|
||||
public final int number;
|
||||
|
||||
|
@ -38,7 +37,6 @@ public final class FieldInfo {
|
|||
public IndexOptions indexOptions;
|
||||
|
||||
public boolean storePayloads; // whether this field stores payloads together with term positions
|
||||
private int codecId = UNASSIGNED_CODEC_ID; // set inside SegmentCodecs#build() during segment flush - this is used to identify the codec used to write this field
|
||||
|
||||
/**
|
||||
* Controls how much information is stored in the postings lists.
|
||||
|
@ -77,21 +75,11 @@ public final class FieldInfo {
|
|||
}
|
||||
assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !storePayloads;
|
||||
}
|
||||
|
||||
void setCodecId(int codecId) {
|
||||
assert this.codecId == UNASSIGNED_CODEC_ID : "CodecId can only be set once.";
|
||||
this.codecId = codecId;
|
||||
}
|
||||
|
||||
public int getCodecId() {
|
||||
return codecId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object clone() {
|
||||
FieldInfo clone = new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
|
||||
clone.codecId = this.codecId;
|
||||
return clone;
|
||||
}
|
||||
|
||||
|
|
|
@ -29,9 +29,6 @@ import java.util.TreeMap;
|
|||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.SegmentCodecs; // Required for Java 1.5 javadocs
|
||||
import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
@ -145,17 +142,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
version = lastVersion = 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new {@link FieldInfos} instance with this as the global field
|
||||
* map
|
||||
*
|
||||
* @return a new {@link FieldInfos} instance with this as the global field
|
||||
* map
|
||||
*/
|
||||
public FieldInfos newFieldInfos(SegmentCodecsBuilder segmentCodecsBuilder) {
|
||||
return new FieldInfos(this, segmentCodecsBuilder);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the last committed version differs from the
|
||||
* current version, otherwise <code>false</code>
|
||||
|
@ -198,7 +184,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
private final SortedMap<Integer,FieldInfo> byNumber = new TreeMap<Integer,FieldInfo>();
|
||||
private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
|
||||
private final FieldNumberBiMap globalFieldNumbers;
|
||||
private final SegmentCodecsBuilder segmentCodecsBuilder;
|
||||
|
||||
// First used in 2.9; prior to 2.9 there was no format header
|
||||
public static final int FORMAT_START = -2;
|
||||
|
@ -230,16 +215,15 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
|
||||
/**
|
||||
* Creates a new {@link FieldInfos} instance with a private
|
||||
* {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap} and a default {@link SegmentCodecsBuilder}
|
||||
* initialized with {@link CodecProvider#getDefault()}.
|
||||
* {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap}
|
||||
* <p>
|
||||
* Note: this ctor should not be used during indexing use
|
||||
* {@link FieldInfos#FieldInfos(FieldInfos)} or
|
||||
* {@link FieldInfos#FieldInfos(FieldNumberBiMap,org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder)}
|
||||
* {@link FieldInfos#FieldInfos(FieldNumberBiMap)}
|
||||
* instead.
|
||||
*/
|
||||
public FieldInfos() {
|
||||
this(new FieldNumberBiMap(), SegmentCodecsBuilder.create(CodecProvider.getDefault()));
|
||||
this(new FieldNumberBiMap());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -249,7 +233,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
* @see #isReadOnly()
|
||||
*/
|
||||
FieldInfos(FieldInfos other) {
|
||||
this(other.globalFieldNumbers, other.segmentCodecsBuilder);
|
||||
this(other.globalFieldNumbers);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -257,9 +241,8 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
* If the {@link FieldNumberBiMap} is <code>null</code> this instance will be read-only.
|
||||
* @see #isReadOnly()
|
||||
*/
|
||||
FieldInfos(FieldNumberBiMap globalFieldNumbers, SegmentCodecsBuilder segmentCodecsBuilder) {
|
||||
FieldInfos(FieldNumberBiMap globalFieldNumbers) {
|
||||
this.globalFieldNumbers = globalFieldNumbers;
|
||||
this.segmentCodecsBuilder = segmentCodecsBuilder;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -273,7 +256,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
* @throws IOException
|
||||
*/
|
||||
public FieldInfos(Directory d, String name) throws IOException {
|
||||
this((FieldNumberBiMap)null, null); // use null here to make this FIs Read-Only
|
||||
this((FieldNumberBiMap)null); // use null here to make this FIs Read-Only
|
||||
final IndexInput input = d.openInput(name, IOContext.READONCE);
|
||||
try {
|
||||
read(input, name);
|
||||
|
@ -309,7 +292,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
*/
|
||||
@Override
|
||||
synchronized public Object clone() {
|
||||
FieldInfos fis = new FieldInfos(globalFieldNumbers, segmentCodecsBuilder);
|
||||
FieldInfos fis = new FieldInfos(globalFieldNumbers);
|
||||
fis.format = format;
|
||||
fis.hasFreq = hasFreq;
|
||||
fis.hasProx = hasProx;
|
||||
|
@ -468,7 +451,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
if (globalFieldNumbers == null) {
|
||||
throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos");
|
||||
}
|
||||
assert segmentCodecsBuilder != null : "SegmentCodecsBuilder is set to null but FieldInfos is not read-only";
|
||||
FieldInfo fi = fieldInfo(name);
|
||||
if (fi == null) {
|
||||
final int fieldNumber = nextFieldNumber(name, preferredFieldNumber);
|
||||
|
@ -477,9 +459,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions);
|
||||
fi.setDocValues(docValues);
|
||||
}
|
||||
if ((fi.isIndexed || fi.hasDocValues()) && fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
|
||||
segmentCodecsBuilder.tryAddAndSet(fi);
|
||||
}
|
||||
version++;
|
||||
return fi;
|
||||
}
|
||||
|
@ -569,22 +548,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds the {@link SegmentCodecs} mapping for this {@link FieldInfos} instance.
|
||||
* @param clearBuilder <code>true</code> iff the internal {@link SegmentCodecsBuilder} must be cleared otherwise <code>false</code>
|
||||
*/
|
||||
public SegmentCodecs buildSegmentCodecs(boolean clearBuilder) {
|
||||
if (globalFieldNumbers == null) {
|
||||
throw new IllegalStateException("FieldInfos are read-only no SegmentCodecs available");
|
||||
}
|
||||
assert segmentCodecsBuilder != null;
|
||||
final SegmentCodecs segmentCodecs = segmentCodecsBuilder.build();
|
||||
if (clearBuilder) {
|
||||
segmentCodecsBuilder.clear();
|
||||
}
|
||||
return segmentCodecs;
|
||||
}
|
||||
|
||||
public void write(Directory d, String name) throws IOException {
|
||||
IndexOutput output = d.createOutput(name, IOContext.READONCE);
|
||||
|
@ -628,7 +591,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
bits |= OMIT_POSITIONS;
|
||||
output.writeString(fi.name);
|
||||
output.writeInt(fi.number);
|
||||
output.writeInt(fi.getCodecId());
|
||||
output.writeByte(bits);
|
||||
|
||||
final byte b;
|
||||
|
@ -698,9 +660,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
|
||||
for (int i = 0; i < size; i++) {
|
||||
String name = input.readString();
|
||||
// if this is a previous format codec 0 will be preflex!
|
||||
final int fieldNumber = format <= FORMAT_FLEX? input.readInt():i;
|
||||
final int codecId = format <= FORMAT_FLEX? input.readInt():0;
|
||||
byte bits = input.readByte();
|
||||
boolean isIndexed = (bits & IS_INDEXED) != 0;
|
||||
boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
|
||||
|
@ -781,8 +741,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
throw new IllegalStateException("unhandled indexValues type " + b);
|
||||
}
|
||||
}
|
||||
final FieldInfo addInternal = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValuesType);
|
||||
addInternal.setCodecId(codecId);
|
||||
addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValuesType);
|
||||
}
|
||||
|
||||
if (input.getFilePointer() != input.length()) {
|
||||
|
@ -804,7 +763,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
if (isReadOnly()) {
|
||||
return this;
|
||||
}
|
||||
final FieldInfos roFis = new FieldInfos((FieldNumberBiMap)null, null);
|
||||
final FieldInfos roFis = new FieldInfos((FieldNumberBiMap)null);
|
||||
for (FieldInfo fieldInfo : this) {
|
||||
FieldInfo clone = (FieldInfo) (fieldInfo).clone();
|
||||
roFis.putInternal(clone);
|
||||
|
@ -814,5 +773,14 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
}
|
||||
return roFis;
|
||||
}
|
||||
|
||||
|
||||
public boolean anyDocValuesFields() {
|
||||
for (FieldInfo fi : this) {
|
||||
if (fi.hasDocValues()) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,7 +57,7 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
|
|||
// Sort by field name
|
||||
CollectionUtil.quickSort(allFields);
|
||||
|
||||
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
|
||||
final FieldsConsumer consumer = state.codec.postingsFormat().fieldsConsumer(state);
|
||||
|
||||
boolean success = false;
|
||||
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FilenameFilter;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
|
@ -29,7 +28,6 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.NoSuchDirectoryException;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
|
@ -122,8 +120,6 @@ final class IndexFileDeleter {
|
|||
infoStream.println("IFD [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
|
||||
}
|
||||
|
||||
private final FilenameFilter indexFilenameFilter;
|
||||
|
||||
// called only from assert
|
||||
private boolean locked() {
|
||||
return writer == null || Thread.holdsLock(writer);
|
||||
|
@ -138,7 +134,7 @@ final class IndexFileDeleter {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
|
||||
PrintStream infoStream, CodecProvider codecs, IndexWriter writer) throws CorruptIndexException, IOException {
|
||||
PrintStream infoStream, IndexWriter writer) throws CorruptIndexException, IOException {
|
||||
this.infoStream = infoStream;
|
||||
this.writer = writer;
|
||||
|
||||
|
@ -154,7 +150,6 @@ final class IndexFileDeleter {
|
|||
// First pass: walk the files and initialize our ref
|
||||
// counts:
|
||||
long currentGen = segmentInfos.getGeneration();
|
||||
indexFilenameFilter = new IndexFileNameFilter(codecs);
|
||||
|
||||
CommitPoint currentCommitPoint = null;
|
||||
String[] files = null;
|
||||
|
@ -167,7 +162,7 @@ final class IndexFileDeleter {
|
|||
|
||||
for (String fileName : files) {
|
||||
|
||||
if ((indexFilenameFilter.accept(null, fileName)) && !fileName.endsWith("write.lock") && !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
|
||||
if ((IndexFileNameFilter.INSTANCE.accept(null, fileName)) && !fileName.endsWith("write.lock") && !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
|
||||
|
||||
// Add this file to refCounts with initial count 0:
|
||||
getRefCount(fileName);
|
||||
|
@ -180,9 +175,9 @@ final class IndexFileDeleter {
|
|||
if (infoStream != null) {
|
||||
message("init: load commit \"" + fileName + "\"");
|
||||
}
|
||||
SegmentInfos sis = new SegmentInfos(codecs);
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
try {
|
||||
sis.read(directory, fileName, codecs);
|
||||
sis.read(directory, fileName);
|
||||
} catch (FileNotFoundException e) {
|
||||
// LUCENE-948: on NFS (and maybe others), if
|
||||
// you have writers switching back and forth
|
||||
|
@ -253,9 +248,9 @@ final class IndexFileDeleter {
|
|||
// listing was stale (eg when index accessed via NFS
|
||||
// client with stale directory listing cache). So we
|
||||
// try now to explicitly open this commit point:
|
||||
SegmentInfos sis = new SegmentInfos(codecs);
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
try {
|
||||
sis.read(directory, currentSegmentsFile, codecs);
|
||||
sis.read(directory, currentSegmentsFile);
|
||||
} catch (IOException e) {
|
||||
throw new CorruptIndexException("failed to locate current segments_N file");
|
||||
}
|
||||
|
@ -373,7 +368,7 @@ final class IndexFileDeleter {
|
|||
for(int i=0;i<files.length;i++) {
|
||||
String fileName = files[i];
|
||||
if ((segmentName == null || fileName.startsWith(segmentPrefix1) || fileName.startsWith(segmentPrefix2)) &&
|
||||
indexFilenameFilter.accept(null, fileName) &&
|
||||
IndexFileNameFilter.INSTANCE.accept(null, fileName) &&
|
||||
!refCounts.containsKey(fileName) &&
|
||||
!fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
|
||||
// Unreferenced file, so remove it
|
||||
|
|
|
@ -20,50 +20,43 @@ package org.apache.lucene.index;
|
|||
import java.io.File;
|
||||
import java.io.FilenameFilter;
|
||||
import java.util.HashSet;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
* Filename filter that accept filenames and extensions only
|
||||
* created by Lucene.
|
||||
* Filename filter that attempts to accept only filenames
|
||||
* created by Lucene. Note that this is a "best effort"
|
||||
* process. If a file is used in a Lucene index, it will
|
||||
* always match the file; but if a file is not used in a
|
||||
* Lucene index but is named in a similar way to Lucene's
|
||||
* files then this filter may accept the file.
|
||||
*
|
||||
* <p>This does not accept <code>*-write.lock</code> files.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
|
||||
public class IndexFileNameFilter implements FilenameFilter {
|
||||
|
||||
private final HashSet<String> extensions;
|
||||
|
||||
public IndexFileNameFilter(CodecProvider codecs) {
|
||||
extensions = new HashSet<String>();
|
||||
for (String ext : IndexFileNames.INDEX_EXTENSIONS) {
|
||||
extensions.add(ext);
|
||||
}
|
||||
if (codecs != null) {
|
||||
for(String ext : codecs.getAllExtensions()) {
|
||||
extensions.add(ext);
|
||||
}
|
||||
}
|
||||
public static final FilenameFilter INSTANCE = new IndexFileNameFilter();
|
||||
|
||||
private IndexFileNameFilter() {
|
||||
}
|
||||
|
||||
// Approximate match for files that seem to be Lucene
|
||||
// index files. This can easily over-match, ie if some
|
||||
// app names a file _foo_bar.go:
|
||||
private final Pattern luceneFilePattern = Pattern.compile("^_[a-z0-9]+(_[a-z0-9]+)?\\.[a-z0-9]+$");
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.io.FilenameFilter#accept(java.io.File, java.lang.String)
|
||||
*/
|
||||
public boolean accept(File dir, String name) {
|
||||
int i = name.lastIndexOf('.');
|
||||
if (i != -1) {
|
||||
String extension = name.substring(1+i);
|
||||
if (extensions.contains(extension)) {
|
||||
return true;
|
||||
} else if (extension.startsWith("f") &&
|
||||
extension.matches("f\\d+")) {
|
||||
return true;
|
||||
} else if (extension.startsWith("s") &&
|
||||
extension.matches("s\\d+")) {
|
||||
return true;
|
||||
}
|
||||
if (name.lastIndexOf('.') != -1) {
|
||||
// Has an extension
|
||||
return luceneFilePattern.matcher(name).matches();
|
||||
} else {
|
||||
if (name.startsWith(IndexFileNames.SEGMENTS)) return true;
|
||||
// No extension -- only segments_N file;
|
||||
return name.startsWith(IndexFileNames.SEGMENTS);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,9 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec; // for javadocs
|
||||
import org.apache.lucene.index.codecs.PostingsFormat; // for javadocs
|
||||
|
||||
// TODO: put all files under codec and remove all the static extensions here
|
||||
|
||||
/**
|
||||
* This class contains useful constants representing filenames and extensions
|
||||
|
@ -31,7 +33,7 @@ import org.apache.lucene.index.codecs.Codec; // for javadocs
|
|||
* {@link #segmentFileName(String, String, String) segmentFileName}).
|
||||
*
|
||||
* <p><b>NOTE</b>: extensions used by codecs are not
|
||||
* listed here. You must interact with the {@link Codec}
|
||||
* listed here. You must interact with the {@link PostingsFormat}
|
||||
* directly.
|
||||
*
|
||||
* @lucene.internal
|
||||
|
@ -188,20 +190,20 @@ public final class IndexFileNames {
|
|||
* <b>NOTE:</b> .<ext> is added to the result file name only if
|
||||
* <code>ext</code> is not empty.
|
||||
* <p>
|
||||
* <b>NOTE:</b> _<name> is added to the result file name only if
|
||||
* <code>name</code> is not empty.
|
||||
* <b>NOTE:</b> _<segmentSuffix> is added to the result file name only if
|
||||
* it's not the empty string
|
||||
* <p>
|
||||
* <b>NOTE:</b> all custom files should be named using this method, or
|
||||
* otherwise some structures may fail to handle them properly (such as if they
|
||||
* are added to compound files).
|
||||
*/
|
||||
public static String segmentFileName(String segmentName, String name, String ext) {
|
||||
if (ext.length() > 0 || name.length() > 0) {
|
||||
public static String segmentFileName(String segmentName, String segmentSuffix, String ext) {
|
||||
if (ext.length() > 0 || segmentSuffix.length() > 0) {
|
||||
assert !ext.startsWith(".");
|
||||
StringBuilder sb = new StringBuilder(segmentName.length() + 2 + name.length() + ext.length());
|
||||
StringBuilder sb = new StringBuilder(segmentName.length() + 2 + segmentSuffix.length() + ext.length());
|
||||
sb.append(segmentName);
|
||||
if (name.length() > 0) {
|
||||
sb.append('_').append(name);
|
||||
if (segmentSuffix.length() > 0) {
|
||||
sb.append('_').append(segmentSuffix);
|
||||
}
|
||||
if (ext.length() > 0) {
|
||||
sb.append('.').append(ext);
|
||||
|
@ -212,11 +214,6 @@ public final class IndexFileNames {
|
|||
}
|
||||
}
|
||||
|
||||
/** Sugar for passing "" + name instead */
|
||||
public static String segmentFileName(String segmentName, int name, String ext) {
|
||||
return segmentFileName(segmentName, ""+name, ext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the given filename ends with the given extension. One
|
||||
* should provide a <i>pure</i> extension, without '.'.
|
||||
|
|
|
@ -28,8 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.search.FieldCache; // javadocs
|
||||
|
@ -312,7 +311,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final Directory directory) throws CorruptIndexException, IOException {
|
||||
return open(directory, null, null, true, DEFAULT_TERMS_INDEX_DIVISOR, null);
|
||||
return open(directory, null, null, true, DEFAULT_TERMS_INDEX_DIVISOR);
|
||||
}
|
||||
|
||||
/** Returns an IndexReader reading the index in the given
|
||||
|
@ -326,9 +325,9 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final Directory directory, boolean readOnly) throws CorruptIndexException, IOException {
|
||||
return open(directory, null, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, null);
|
||||
return open(directory, null, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Open a near real time IndexReader from the {@link org.apache.lucene.index.IndexWriter}.
|
||||
*
|
||||
|
@ -363,7 +362,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final IndexCommit commit, boolean readOnly) throws CorruptIndexException, IOException {
|
||||
return open(commit.getDirectory(), null, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, null);
|
||||
return open(commit.getDirectory(), null, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
|
||||
}
|
||||
|
||||
/** Expert: returns an IndexReader reading the index in
|
||||
|
@ -381,7 +380,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final Directory directory, IndexDeletionPolicy deletionPolicy, boolean readOnly) throws CorruptIndexException, IOException {
|
||||
return open(directory, deletionPolicy, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, null);
|
||||
return open(directory, deletionPolicy, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
|
||||
}
|
||||
|
||||
/** Expert: returns an IndexReader reading the index in
|
||||
|
@ -409,7 +408,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final Directory directory, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
|
||||
return open(directory, deletionPolicy, null, readOnly, termInfosIndexDivisor, null);
|
||||
return open(directory, deletionPolicy, null, readOnly, termInfosIndexDivisor);
|
||||
}
|
||||
|
||||
/** Expert: returns an IndexReader reading the index in
|
||||
|
@ -429,7 +428,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final IndexCommit commit, IndexDeletionPolicy deletionPolicy, boolean readOnly) throws CorruptIndexException, IOException {
|
||||
return open(commit.getDirectory(), deletionPolicy, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, null);
|
||||
return open(commit.getDirectory(), deletionPolicy, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
|
||||
}
|
||||
|
||||
/** Expert: returns an IndexReader reading the index in
|
||||
|
@ -462,78 +461,11 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final IndexCommit commit, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
|
||||
return open(commit.getDirectory(), deletionPolicy, commit, readOnly, termInfosIndexDivisor, null);
|
||||
return open(commit.getDirectory(), deletionPolicy, commit, readOnly, termInfosIndexDivisor);
|
||||
}
|
||||
|
||||
/** Expert: returns an IndexReader reading the index in
|
||||
* the given Directory, with a custom {@link
|
||||
* IndexDeletionPolicy}, and specified {@link CodecProvider}.
|
||||
* You should pass readOnly=true, since it gives much
|
||||
* better concurrent performance, unless you intend to do
|
||||
* write operations (delete documents or change norms)
|
||||
* with the reader.
|
||||
* @param directory the index directory
|
||||
* @param deletionPolicy a custom deletion policy (only used
|
||||
* if you use this reader to perform deletes or to set
|
||||
* norms); see {@link IndexWriter} for details.
|
||||
* @param readOnly true if no changes (deletions, norms) will be made with this IndexReader
|
||||
* @param termInfosIndexDivisor Subsamples which indexed
|
||||
* terms are loaded into RAM. This has the same effect as {@link
|
||||
* IndexWriterConfig#setTermIndexInterval} except that setting
|
||||
* must be done at indexing time while this setting can be
|
||||
* set per reader. When set to N, then one in every
|
||||
* N*termIndexInterval terms in the index is loaded into
|
||||
* memory. By setting this to a value > 1 you can reduce
|
||||
* memory usage, at the expense of higher latency when
|
||||
* loading a TermInfo. The default value is 1. Set this
|
||||
* to -1 to skip loading the terms index entirely.
|
||||
* @param codecs CodecProvider to use when opening index
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final Directory directory, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor, CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
return open(directory, deletionPolicy, null, readOnly, termInfosIndexDivisor, codecs);
|
||||
}
|
||||
|
||||
/** Expert: returns an IndexReader reading the index in
|
||||
* the given Directory, using a specific commit and with
|
||||
* a custom {@link IndexDeletionPolicy} and specified
|
||||
* {@link CodecProvider}. You should pass readOnly=true, since
|
||||
* it gives much better concurrent performance, unless
|
||||
* you intend to do write operations (delete documents or
|
||||
* change norms) with the reader.
|
||||
|
||||
* @param commit the specific {@link IndexCommit} to open;
|
||||
* see {@link IndexReader#listCommits} to list all commits
|
||||
* in a directory
|
||||
* @param deletionPolicy a custom deletion policy (only used
|
||||
* if you use this reader to perform deletes or to set
|
||||
* norms); see {@link IndexWriter} for details.
|
||||
* @param readOnly true if no changes (deletions, norms) will be made with this IndexReader
|
||||
* @param termInfosIndexDivisor Subsamples which indexed
|
||||
* terms are loaded into RAM. This has the same effect as {@link
|
||||
* IndexWriterConfig#setTermIndexInterval} except that setting
|
||||
* must be done at indexing time while this setting can be
|
||||
* set per reader. When set to N, then one in every
|
||||
* N*termIndexInterval terms in the index is loaded into
|
||||
* memory. By setting this to a value > 1 you can reduce
|
||||
* memory usage, at the expense of higher latency when
|
||||
* loading a TermInfo. The default value is 1. Set this
|
||||
* to -1 to skip loading the terms index entirely.
|
||||
* @param codecs CodecProvider to use when opening index
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static IndexReader open(final IndexCommit commit, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor, CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
return open(commit.getDirectory(), deletionPolicy, commit, readOnly, termInfosIndexDivisor, codecs);
|
||||
}
|
||||
|
||||
private static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly, int termInfosIndexDivisor,
|
||||
CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
if (codecs == null) {
|
||||
codecs = CodecProvider.getDefault();
|
||||
}
|
||||
return DirectoryReader.open(directory, deletionPolicy, commit, readOnly, termInfosIndexDivisor, codecs);
|
||||
private static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
|
||||
return DirectoryReader.open(directory, deletionPolicy, commit, readOnly, termInfosIndexDivisor);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -767,7 +699,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
}
|
||||
}.run()).longValue();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Reads version number from segments files. The version number is
|
||||
* initialized with a timestamp and then increased by one for each change of
|
||||
|
@ -779,24 +711,9 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static long getCurrentVersion(Directory directory) throws CorruptIndexException, IOException {
|
||||
return getCurrentVersion(directory, CodecProvider.getDefault());
|
||||
return SegmentInfos.readCurrentVersion(directory);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads version number from segments files. The version number is
|
||||
* initialized with a timestamp and then increased by one for each change of
|
||||
* the index.
|
||||
*
|
||||
* @param directory where the index resides.
|
||||
* @param codecs the {@link CodecProvider} holding all {@link Codec}s required to open the index
|
||||
* @return version number.
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static long getCurrentVersion(Directory directory, CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
return SegmentInfos.readCurrentVersion(directory, codecs);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads commitUserData, previously passed to {@link
|
||||
* IndexWriter#commit(Map)}, from current index
|
||||
|
@ -811,28 +728,8 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
*
|
||||
* @see #getCommitUserData()
|
||||
*/
|
||||
public static Map<String,String> getCommitUserData(Directory directory) throws CorruptIndexException, IOException {
|
||||
return getCommitUserData(directory, CodecProvider.getDefault());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Reads commitUserData, previously passed to {@link
|
||||
* IndexWriter#commit(Map)}, from current index
|
||||
* segments file. This will return null if {@link
|
||||
* IndexWriter#commit(Map)} has never been called for
|
||||
* this index.
|
||||
*
|
||||
* @param directory where the index resides.
|
||||
* @param codecs the {@link CodecProvider} provider holding all {@link Codec}s required to open the index
|
||||
* @return commit userData.
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*
|
||||
* @see #getCommitUserData()
|
||||
*/
|
||||
public static Map<String, String> getCommitUserData(Directory directory, CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
return SegmentInfos.readCurrentUserData(directory, codecs);
|
||||
public static Map<String, String> getCommitUserData(Directory directory) throws CorruptIndexException, IOException {
|
||||
return SegmentInfos.readCurrentUserData(directory);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -985,22 +882,6 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if an index exists at the specified directory.
|
||||
* @param directory the directory to check for an index
|
||||
* @param codecProvider provides a CodecProvider in case the index uses non-core codecs
|
||||
* @return <code>true</code> if an index exists; <code>false</code> otherwise
|
||||
* @throws IOException if there is a problem with accessing the index
|
||||
*/
|
||||
public static boolean indexExists(Directory directory, CodecProvider codecProvider) throws IOException {
|
||||
try {
|
||||
new SegmentInfos().read(directory, codecProvider);
|
||||
return true;
|
||||
} catch (IOException ioe) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns the number of documents in this index. */
|
||||
public abstract int numDocs();
|
||||
|
||||
|
|
|
@ -114,12 +114,12 @@ public final class IndexUpgrader {
|
|||
}
|
||||
|
||||
public void upgrade() throws IOException {
|
||||
if (!IndexReader.indexExists(dir, iwc.getCodecProvider())) {
|
||||
if (!IndexReader.indexExists(dir)) {
|
||||
throw new IndexNotFoundException(dir.toString());
|
||||
}
|
||||
|
||||
if (!deletePriorCommits) {
|
||||
final Collection<IndexCommit> commits = DirectoryReader.listCommits(dir, iwc.getCodecProvider());
|
||||
final Collection<IndexCommit> commits = DirectoryReader.listCommits(dir);
|
||||
if (commits.size() > 1) {
|
||||
throw new IllegalArgumentException("This tool was invoked to not delete prior commit points, but the following commits were found: " + commits);
|
||||
}
|
||||
|
|
|
@ -39,8 +39,7 @@ import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
|
|||
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
|
||||
import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
|
@ -375,7 +374,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// just like we do when loading segments_N
|
||||
synchronized(this) {
|
||||
maybeApplyDeletes(applyAllDeletes);
|
||||
r = new DirectoryReader(this, segmentInfos, codecs, applyAllDeletes);
|
||||
r = new DirectoryReader(this, segmentInfos, applyAllDeletes);
|
||||
if (infoStream != null) {
|
||||
message("return reader version=" + r.getVersion() + " reader=" + r);
|
||||
}
|
||||
|
@ -802,7 +801,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
infoStream.println("IW " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
|
||||
}
|
||||
|
||||
CodecProvider codecs;
|
||||
final Codec codec; // for writing new segments
|
||||
|
||||
/**
|
||||
* Constructs a new IndexWriter per the settings given in <code>conf</code>.
|
||||
|
@ -837,7 +836,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
mergePolicy = conf.getMergePolicy();
|
||||
mergePolicy.setIndexWriter(this);
|
||||
mergeScheduler = conf.getMergeScheduler();
|
||||
codecs = conf.getCodecProvider();
|
||||
codec = conf.getCodec();
|
||||
|
||||
bufferedDeletesStream = new BufferedDeletesStream(messageID);
|
||||
bufferedDeletesStream.setInfoStream(infoStream);
|
||||
|
@ -862,7 +861,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// If index is too old, reading the segments will throw
|
||||
// IndexFormatTooOldException.
|
||||
segmentInfos = new SegmentInfos(codecs);
|
||||
segmentInfos = new SegmentInfos();
|
||||
try {
|
||||
if (create) {
|
||||
// Try to read first. This is to allow create
|
||||
|
@ -870,7 +869,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// searching. In this case we write the next
|
||||
// segments_N file with no segments:
|
||||
try {
|
||||
segmentInfos.read(directory, codecs);
|
||||
segmentInfos.read(directory);
|
||||
segmentInfos.clear();
|
||||
} catch (IOException e) {
|
||||
// Likely this means it's a fresh directory
|
||||
|
@ -881,7 +880,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
changeCount++;
|
||||
segmentInfos.changed();
|
||||
} else {
|
||||
segmentInfos.read(directory, codecs);
|
||||
segmentInfos.read(directory);
|
||||
|
||||
IndexCommit commit = conf.getIndexCommit();
|
||||
if (commit != null) {
|
||||
|
@ -892,8 +891,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// points.
|
||||
if (commit.getDirectory() != directory)
|
||||
throw new IllegalArgumentException("IndexCommit's directory doesn't match my directory");
|
||||
SegmentInfos oldInfos = new SegmentInfos(codecs);
|
||||
oldInfos.read(directory, commit.getSegmentsFileName(), codecs);
|
||||
SegmentInfos oldInfos = new SegmentInfos();
|
||||
oldInfos.read(directory, commit.getSegmentsFileName());
|
||||
segmentInfos.replace(oldInfos);
|
||||
changeCount++;
|
||||
segmentInfos.changed();
|
||||
|
@ -906,7 +905,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// start with previous field numbers, but new FieldInfos
|
||||
globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory);
|
||||
docWriter = new DocumentsWriter(config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
|
||||
docWriter = new DocumentsWriter(codec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
|
||||
docWriter.setInfoStream(infoStream);
|
||||
|
||||
// Default deleter (for backwards compatibility) is
|
||||
|
@ -914,8 +913,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
synchronized(this) {
|
||||
deleter = new IndexFileDeleter(directory,
|
||||
conf.getIndexDeletionPolicy(),
|
||||
segmentInfos, infoStream,
|
||||
codecs, this);
|
||||
segmentInfos, infoStream, this);
|
||||
}
|
||||
|
||||
if (deleter.startingCommitDeleted) {
|
||||
|
@ -2149,6 +2147,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
*/
|
||||
public synchronized void deleteAll() throws IOException {
|
||||
ensureOpen();
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
// Abort any running merges
|
||||
|
@ -2170,10 +2169,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// Mark that the index has changed
|
||||
++changeCount;
|
||||
segmentInfos.changed();
|
||||
success = true;
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteAll");
|
||||
} finally {
|
||||
if (infoStream != null) {
|
||||
if (!success && infoStream != null) {
|
||||
message("hit exception during deleteAll");
|
||||
}
|
||||
}
|
||||
|
@ -2476,8 +2476,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (infoStream != null) {
|
||||
message("addIndexes: process directory " + dir);
|
||||
}
|
||||
SegmentInfos sis = new SegmentInfos(codecs); // read infos from dir
|
||||
sis.read(dir, codecs);
|
||||
SegmentInfos sis = new SegmentInfos(); // read infos from dir
|
||||
sis.read(dir);
|
||||
final Set<String> dsFilesCopied = new HashSet<String>();
|
||||
final Map<String, String> dsNames = new HashMap<String, String>();
|
||||
for (SegmentInfo info : sis) {
|
||||
|
@ -2567,7 +2567,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// abortable so that IW.close(false) is able to stop it
|
||||
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(),
|
||||
mergedName, null, payloadProcessorProvider,
|
||||
globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)), context);
|
||||
new FieldInfos(globalFieldNumberMap), codec, context);
|
||||
|
||||
for (IndexReader reader : readers) // add new indexes
|
||||
merger.add(reader);
|
||||
|
@ -2575,7 +2575,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final FieldInfos fieldInfos = merger.fieldInfos();
|
||||
SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
|
||||
false, merger.getSegmentCodecs(),
|
||||
false, merger.getCodec(),
|
||||
fieldInfos);
|
||||
setDiagnostics(info, "addIndexes(IndexReader...)");
|
||||
|
||||
|
@ -2591,7 +2591,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// Now create the compound file if needed
|
||||
if (useCompoundFile) {
|
||||
merger.createCompoundFile(mergedName + ".cfs", info, context);
|
||||
merger.createCompoundFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION), info, context);
|
||||
|
||||
// delete new non cfs files directly: they were never
|
||||
// registered with IFD
|
||||
|
@ -2916,7 +2916,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
try {
|
||||
if (infoStream != null)
|
||||
message("commit: pendingCommit != null");
|
||||
pendingCommit.finishCommit(directory);
|
||||
pendingCommit.finishCommit(directory, codec);
|
||||
if (infoStream != null)
|
||||
message("commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName() + "\"");
|
||||
lastCommitChangeCount = pendingCommitChangeCount;
|
||||
|
@ -3459,7 +3459,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// Bind a new segment name here so even with
|
||||
// ConcurrentMergePolicy we keep deterministic segment
|
||||
// names.
|
||||
merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, null, globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)));
|
||||
merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, null, new FieldInfos(globalFieldNumberMap));
|
||||
|
||||
// Lock order: IW -> BD
|
||||
final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, merge.segments);
|
||||
|
@ -3633,7 +3633,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
IOContext context = new IOContext(merge.getMergeInfo());
|
||||
|
||||
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName, merge,
|
||||
payloadProcessorProvider, merge.info.getFieldInfos(), context);
|
||||
payloadProcessorProvider, merge.info.getFieldInfos(), codec, context);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
|
||||
|
@ -3679,10 +3679,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
mergedDocCount = merge.info.docCount = merger.merge();
|
||||
|
||||
// Record which codec was used to write the segment
|
||||
merge.info.setSegmentCodecs(merger.getSegmentCodecs());
|
||||
merge.info.setCodec(merger.getCodec());
|
||||
|
||||
if (infoStream != null) {
|
||||
message("merge segmentCodecs=" + merger.getSegmentCodecs());
|
||||
message("merge codecs=" + merger.getCodec());
|
||||
message("merge store matchedCount=" + merger.getMatchedSubReaderCount() + " vs " + merge.readers.size());
|
||||
}
|
||||
anyNonBulkMerges |= merger.getAnyNonBulkMerges();
|
||||
|
@ -3975,7 +3975,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// Exception here means nothing is prepared
|
||||
// (this method unwinds everything it did on
|
||||
// an exception)
|
||||
toSync.prepareCommit(directory);
|
||||
toSync.prepareCommit(directory, codec);
|
||||
|
||||
pendingCommitSet = true;
|
||||
pendingCommit = toSync;
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.index;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
|
||||
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
@ -121,7 +121,7 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
private volatile int maxBufferedDocs;
|
||||
private volatile IndexingChain indexingChain;
|
||||
private volatile IndexReaderWarmer mergedSegmentWarmer;
|
||||
private volatile CodecProvider codecProvider;
|
||||
private volatile Codec codec;
|
||||
private volatile MergePolicy mergePolicy;
|
||||
private volatile DocumentsWriterPerThreadPool indexerThreadPool;
|
||||
private volatile boolean readerPooling;
|
||||
|
@ -158,7 +158,7 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
maxBufferedDocs = DEFAULT_MAX_BUFFERED_DOCS;
|
||||
indexingChain = DocumentsWriterPerThread.defaultIndexingChain;
|
||||
mergedSegmentWarmer = null;
|
||||
codecProvider = CodecProvider.getDefault();
|
||||
codec = Codec.getDefault();
|
||||
if (matchVersion.onOrAfter(Version.LUCENE_32)) {
|
||||
mergePolicy = new TieredMergePolicy();
|
||||
} else {
|
||||
|
@ -521,17 +521,17 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
return this;
|
||||
}
|
||||
|
||||
/** Set the CodecProvider. See {@link CodecProvider}.
|
||||
/** Set the Codec. See {@link Codec}.
|
||||
*
|
||||
* <p>Only takes effect when IndexWriter is first created. */
|
||||
public IndexWriterConfig setCodecProvider(CodecProvider codecProvider) {
|
||||
this.codecProvider = codecProvider;
|
||||
public IndexWriterConfig setCodec(Codec codec) {
|
||||
this.codec = codec;
|
||||
return this;
|
||||
}
|
||||
|
||||
/** Returns the current merged segment warmer. See {@link IndexReaderWarmer}. */
|
||||
public CodecProvider getCodecProvider() {
|
||||
return codecProvider;
|
||||
/** Returns the current Codec. See {@link Codec}. */
|
||||
public Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
|
||||
|
@ -694,7 +694,7 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
sb.append("ramBufferSizeMB=").append(ramBufferSizeMB).append("\n");
|
||||
sb.append("maxBufferedDocs=").append(maxBufferedDocs).append("\n");
|
||||
sb.append("mergedSegmentWarmer=").append(mergedSegmentWarmer).append("\n");
|
||||
sb.append("codecProvider=").append(codecProvider).append("\n");
|
||||
sb.append("codec=").append(codec).append("\n");
|
||||
sb.append("mergePolicy=").append(mergePolicy).append("\n");
|
||||
sb.append("indexerThreadPool=").append(indexerThreadPool).append("\n");
|
||||
sb.append("readerPooling=").append(readerPooling).append("\n");
|
||||
|
|
|
@ -34,41 +34,37 @@ public class PerDocWriteState {
|
|||
public final String segmentName;
|
||||
public final FieldInfos fieldInfos;
|
||||
public final Counter bytesUsed;
|
||||
public final SegmentCodecs segmentCodecs;
|
||||
public final int codecId;
|
||||
public final String segmentSuffix;
|
||||
public final IOContext context;
|
||||
|
||||
PerDocWriteState(PrintStream infoStream, Directory directory,
|
||||
public PerDocWriteState(PrintStream infoStream, Directory directory,
|
||||
String segmentName, FieldInfos fieldInfos, Counter bytesUsed,
|
||||
int codecId, IOContext context) {
|
||||
String segmentSuffix, IOContext context) {
|
||||
this.infoStream = infoStream;
|
||||
this.directory = directory;
|
||||
this.segmentName = segmentName;
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.segmentCodecs = fieldInfos.buildSegmentCodecs(false);
|
||||
this.codecId = codecId;
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
this.bytesUsed = bytesUsed;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
PerDocWriteState(SegmentWriteState state) {
|
||||
public PerDocWriteState(SegmentWriteState state) {
|
||||
infoStream = state.infoStream;
|
||||
directory = state.directory;
|
||||
segmentCodecs = state.segmentCodecs;
|
||||
segmentName = state.segmentName;
|
||||
fieldInfos = state.fieldInfos;
|
||||
codecId = state.codecId;
|
||||
segmentSuffix = state.segmentSuffix;
|
||||
bytesUsed = Counter.newCounter();
|
||||
context = state.context;
|
||||
}
|
||||
|
||||
PerDocWriteState(PerDocWriteState state, int codecId) {
|
||||
public PerDocWriteState(PerDocWriteState state, String segmentSuffix) {
|
||||
this.infoStream = state.infoStream;
|
||||
this.directory = state.directory;
|
||||
this.segmentName = state.segmentName;
|
||||
this.fieldInfos = state.fieldInfos;
|
||||
this.segmentCodecs = state.segmentCodecs;
|
||||
this.codecId = codecId;
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
this.bytesUsed = state.bytesUsed;
|
||||
this.context = state.context;
|
||||
}
|
||||
|
|
|
@ -1,286 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.TermsConsumer;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Enables native per field codec support. This class selects the codec used to
|
||||
* write a field depending on the provided {@link SegmentCodecs}. For each field
|
||||
* seen it resolves the codec based on the {@link FieldInfo#codecId} which is
|
||||
* only valid during a segment merge. See {@link SegmentCodecs} javadoc for
|
||||
* details.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class PerFieldCodecWrapper extends Codec {
|
||||
private final SegmentCodecs segmentCodecs;
|
||||
|
||||
PerFieldCodecWrapper(SegmentCodecs segmentCodecs) {
|
||||
super("PerField");
|
||||
this.segmentCodecs = segmentCodecs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
|
||||
throws IOException {
|
||||
return new FieldsWriter(state);
|
||||
}
|
||||
|
||||
private class FieldsWriter extends FieldsConsumer {
|
||||
private final ArrayList<FieldsConsumer> consumers = new ArrayList<FieldsConsumer>();
|
||||
|
||||
public FieldsWriter(SegmentWriteState state) throws IOException {
|
||||
assert segmentCodecs == state.segmentCodecs;
|
||||
final Codec[] codecs = segmentCodecs.codecs;
|
||||
for (int i = 0; i < codecs.length; i++) {
|
||||
boolean success = false;
|
||||
try {
|
||||
consumers.add(codecs[i].fieldsConsumer(new SegmentWriteState(state, i)));
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(consumers);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsConsumer addField(FieldInfo field) throws IOException {
|
||||
assert field.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
|
||||
final FieldsConsumer fields = consumers.get(field.getCodecId());
|
||||
return fields.addField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
IOUtils.close(consumers);
|
||||
}
|
||||
}
|
||||
|
||||
private class FieldsReader extends FieldsProducer {
|
||||
|
||||
private final Set<String> fields = new TreeSet<String>();
|
||||
private final Map<String, FieldsProducer> codecs = new HashMap<String, FieldsProducer>();
|
||||
|
||||
public FieldsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
|
||||
IOContext context, int indexDivisor) throws IOException {
|
||||
|
||||
final Map<Codec, FieldsProducer> producers = new HashMap<Codec, FieldsProducer>();
|
||||
boolean success = false;
|
||||
try {
|
||||
for (FieldInfo fi : fieldInfos) {
|
||||
if (fi.isIndexed) {
|
||||
fields.add(fi.name);
|
||||
assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
|
||||
Codec codec = segmentCodecs.codecs[fi.getCodecId()];
|
||||
if (!producers.containsKey(codec)) {
|
||||
producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
|
||||
si, fieldInfos, context, indexDivisor, fi.getCodecId())));
|
||||
}
|
||||
codecs.put(fi.name, producers.get(codec));
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
// If we hit exception (eg, IOE because writer was
|
||||
// committing, or, for any other reason) we must
|
||||
// go back and close all FieldsProducers we opened:
|
||||
IOUtils.closeWhileHandlingException(producers.values());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private final class FieldsIterator extends FieldsEnum {
|
||||
private final Iterator<String> it;
|
||||
private String current;
|
||||
|
||||
public FieldsIterator() {
|
||||
it = fields.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String next() {
|
||||
if (it.hasNext()) {
|
||||
current = it.next();
|
||||
} else {
|
||||
current = null;
|
||||
}
|
||||
|
||||
return current;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() throws IOException {
|
||||
Terms terms = codecs.get(current).terms(current);
|
||||
if (terms != null) {
|
||||
return terms.iterator();
|
||||
} else {
|
||||
return TermsEnum.EMPTY;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsEnum iterator() throws IOException {
|
||||
return new FieldsIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
FieldsProducer fields = codecs.get(field);
|
||||
return fields == null ? null : fields.terms(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
IOUtils.close(codecs.values());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state)
|
||||
throws IOException {
|
||||
return new FieldsReader(state.dir, state.fieldInfos, state.segmentInfo,
|
||||
state.context, state.termsIndexDivisor);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, int codecId, Set<String> files)
|
||||
throws IOException {
|
||||
// ignore codecid since segmentCodec will assign it per codec
|
||||
segmentCodecs.files(dir, info, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
for (Codec codec : segmentCodecs.codecs) {
|
||||
codec.getExtensions(extensions);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new PerDocConsumers(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new PerDocProducers(state.dir, state.fieldInfos, state.segmentInfo,
|
||||
state.context, state.termsIndexDivisor);
|
||||
}
|
||||
|
||||
private final class PerDocProducers extends PerDocValues {
|
||||
private final TreeMap<String, PerDocValues> codecs = new TreeMap<String, PerDocValues>();
|
||||
|
||||
public PerDocProducers(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
|
||||
IOContext context, int indexDivisor) throws IOException {
|
||||
final Map<Codec, PerDocValues> producers = new HashMap<Codec, PerDocValues>();
|
||||
boolean success = false;
|
||||
try {
|
||||
for (FieldInfo fi : fieldInfos) {
|
||||
if (fi.hasDocValues()) {
|
||||
assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
|
||||
Codec codec = segmentCodecs.codecs[fi.getCodecId()];
|
||||
if (!producers.containsKey(codec)) {
|
||||
producers.put(codec, codec.docsProducer(new SegmentReadState(dir,
|
||||
si, fieldInfos, context, indexDivisor, fi.getCodecId())));
|
||||
}
|
||||
codecs.put(fi.name, producers.get(codec));
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(producers.values());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> fields() {
|
||||
return codecs.keySet();
|
||||
}
|
||||
@Override
|
||||
public IndexDocValues docValues(String field) throws IOException {
|
||||
final PerDocValues perDocProducer = codecs.get(field);
|
||||
if (perDocProducer == null) {
|
||||
return null;
|
||||
}
|
||||
return perDocProducer.docValues(field);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.close(codecs.values());
|
||||
}
|
||||
}
|
||||
|
||||
private final class PerDocConsumers extends PerDocConsumer {
|
||||
private final PerDocConsumer[] consumers;
|
||||
private final Codec[] codecs;
|
||||
private final PerDocWriteState state;
|
||||
|
||||
public PerDocConsumers(PerDocWriteState state) throws IOException {
|
||||
assert segmentCodecs == state.segmentCodecs;
|
||||
this.state = state;
|
||||
codecs = segmentCodecs.codecs;
|
||||
consumers = new PerDocConsumer[codecs.length];
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.close(consumers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
|
||||
final int codecId = field.getCodecId();
|
||||
assert codecId != FieldInfo.UNASSIGNED_CODEC_ID;
|
||||
PerDocConsumer perDoc = consumers[codecId];
|
||||
if (perDoc == null) {
|
||||
perDoc = codecs[codecId].docsConsumer(new PerDocWriteState(state, codecId));
|
||||
assert perDoc != null;
|
||||
consumers[codecId] = perDoc;
|
||||
}
|
||||
return perDoc.addValuesField(field);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -1,167 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
||||
/**
|
||||
* SegmentCodecs maintains an ordered list of distinct codecs used within a
|
||||
* segment. Within a segment on codec is used to write multiple fields while
|
||||
* each field could be written by a different codec. To enable codecs per field
|
||||
* within a single segment we need to record the distinct codecs and map them to
|
||||
* each field present in the segment. SegmentCodecs is created together with
|
||||
* {@link SegmentWriteState} for each flush and is maintained in the
|
||||
* corresponding {@link SegmentInfo} until it is committed.
|
||||
* <p>
|
||||
* During indexing {@link FieldInfos} uses {@link SegmentCodecsBuilder} to incrementally
|
||||
* build the {@link SegmentCodecs} mapping. Once a segment is flushed
|
||||
* DocumentsWriter creates a {@link SegmentCodecs} instance from
|
||||
* {@link FieldInfos#buildSegmentCodecs(boolean)} The {@link FieldInfo#codecId}
|
||||
* assigned by {@link SegmentCodecsBuilder} refers to the codecs ordinal
|
||||
* maintained inside {@link SegmentCodecs}. This ord is later used to get the
|
||||
* right codec when the segment is opened in a reader.The {@link Codec} returned
|
||||
* from {@link SegmentCodecs#codec()} in turn uses {@link SegmentCodecs}
|
||||
* internal structure to select and initialize the right codec for a fields when
|
||||
* it is written.
|
||||
* <p>
|
||||
* Once a flush succeeded the {@link SegmentCodecs} is maintained inside the
|
||||
* {@link SegmentInfo} for the flushed segment it was created for.
|
||||
* {@link SegmentInfo} writes the name of each codec in {@link SegmentCodecs}
|
||||
* for each segment and maintains the order. Later if a segment is opened by a
|
||||
* reader this mapping is deserialized and used to create the codec per field.
|
||||
*
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class SegmentCodecs implements Cloneable {
|
||||
/**
|
||||
* internal structure to map codecs to fields - don't modify this from outside
|
||||
* of this class!
|
||||
*/
|
||||
final Codec[] codecs;
|
||||
final CodecProvider provider;
|
||||
private final Codec codec;
|
||||
|
||||
SegmentCodecs(CodecProvider provider, IndexInput input) throws IOException {
|
||||
this(provider, read(input, provider));
|
||||
}
|
||||
|
||||
SegmentCodecs(CodecProvider provider, Codec... codecs) {
|
||||
this.provider = provider;
|
||||
this.codecs = codecs;
|
||||
if (codecs.length == 1 && codecs[0] instanceof PreFlexCodec) {
|
||||
this.codec = codecs[0]; // hack for backwards break... don't wrap the codec in preflex
|
||||
} else {
|
||||
this.codec = new PerFieldCodecWrapper(this);
|
||||
}
|
||||
}
|
||||
|
||||
Codec codec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
void write(IndexOutput out) throws IOException {
|
||||
out.writeVInt(codecs.length);
|
||||
for (Codec codec : codecs) {
|
||||
out.writeString(codec.name);
|
||||
}
|
||||
}
|
||||
|
||||
private static Codec[] read(IndexInput in, CodecProvider provider) throws IOException {
|
||||
final int size = in.readVInt();
|
||||
final ArrayList<Codec> list = new ArrayList<Codec>();
|
||||
for (int i = 0; i < size; i++) {
|
||||
final String codecName = in.readString();
|
||||
final Codec lookup = provider.lookup(codecName);
|
||||
list.add(i, lookup);
|
||||
}
|
||||
return list.toArray(Codec.EMPTY);
|
||||
}
|
||||
|
||||
void files(Directory dir, SegmentInfo info, Set<String> files)
|
||||
throws IOException {
|
||||
final Codec[] codecArray = codecs;
|
||||
for (int i = 0; i < codecArray.length; i++) {
|
||||
codecArray[i].files(dir, info, i, files);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SegmentCodecs [codecs=" + Arrays.toString(codecs) + ", provider=" + provider + "]";
|
||||
}
|
||||
|
||||
/**
|
||||
* Used in {@link FieldInfos} to incrementally build the codec ID mapping for
|
||||
* {@link FieldInfo} instances.
|
||||
* <p>
|
||||
* Note: this class is not thread-safe
|
||||
* </p>
|
||||
* @see FieldInfo#getCodecId()
|
||||
*/
|
||||
final static class SegmentCodecsBuilder {
|
||||
private final Map<Codec, Integer> codecRegistry = new IdentityHashMap<Codec, Integer>();
|
||||
private final ArrayList<Codec> codecs = new ArrayList<Codec>();
|
||||
private final CodecProvider provider;
|
||||
|
||||
private SegmentCodecsBuilder(CodecProvider provider) {
|
||||
this.provider = provider;
|
||||
}
|
||||
|
||||
static SegmentCodecsBuilder create(CodecProvider provider) {
|
||||
return new SegmentCodecsBuilder(provider);
|
||||
}
|
||||
|
||||
SegmentCodecsBuilder tryAddAndSet(FieldInfo fi) {
|
||||
if (fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
|
||||
final Codec fieldCodec = provider.lookup(provider
|
||||
.getFieldCodec(fi.name));
|
||||
Integer ord = codecRegistry.get(fieldCodec);
|
||||
if (ord == null) {
|
||||
ord = Integer.valueOf(codecs.size());
|
||||
codecRegistry.put(fieldCodec, ord);
|
||||
codecs.add(fieldCodec);
|
||||
}
|
||||
fi.setCodecId(ord.intValue());
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
SegmentCodecs build() {
|
||||
return new SegmentCodecs(provider, codecs.toArray(Codec.EMPTY));
|
||||
}
|
||||
|
||||
SegmentCodecsBuilder clear() {
|
||||
codecRegistry.clear();
|
||||
codecs.clear();
|
||||
return this;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
|
@ -68,7 +69,7 @@ final class SegmentCoreReaders {
|
|||
}
|
||||
|
||||
segment = si.name;
|
||||
final SegmentCodecs segmentCodecs = si.getSegmentCodecs();
|
||||
final Codec codec = si.getCodec();
|
||||
this.context = context;
|
||||
this.dir = dir;
|
||||
|
||||
|
@ -85,12 +86,12 @@ final class SegmentCoreReaders {
|
|||
fieldInfos = si.getFieldInfos();
|
||||
|
||||
this.termsIndexDivisor = termsIndexDivisor;
|
||||
final Codec codec = segmentCodecs.codec();
|
||||
final PostingsFormat format = codec.postingsFormat();
|
||||
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si, fieldInfos, context, termsIndexDivisor);
|
||||
// Ask codec for its Fields
|
||||
fields = codec.fieldsProducer(segmentReadState);
|
||||
fields = format.fieldsProducer(segmentReadState);
|
||||
assert fields != null;
|
||||
perDocProducer = codec.docsProducer(segmentReadState);
|
||||
perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -165,7 +166,7 @@ final class SegmentCoreReaders {
|
|||
}
|
||||
|
||||
final String storesSegment = si.getDocStoreSegment();
|
||||
fieldsReaderOrig = si.getSegmentCodecs().provider.fieldsReader(storeDir, storesSegment, fieldInfos, context,
|
||||
fieldsReaderOrig = si.getCodec().fieldsFormat().fieldsReader(storeDir, storesSegment, fieldInfos, context,
|
||||
si.getDocStoreOffset(), si.docCount);
|
||||
|
||||
// Verify two sources of "maxDoc" agree:
|
||||
|
|
|
@ -28,7 +28,6 @@ import java.util.Map.Entry;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -97,7 +96,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
|
||||
private FieldInfos fieldInfos;
|
||||
|
||||
private SegmentCodecs segmentCodecs;
|
||||
private Codec codec;
|
||||
|
||||
private Map<String,String> diagnostics;
|
||||
|
||||
|
@ -116,7 +115,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
private long fieldInfosVersion;
|
||||
|
||||
public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
|
||||
SegmentCodecs segmentCodecs, FieldInfos fieldInfos) {
|
||||
Codec codec, FieldInfos fieldInfos) {
|
||||
this.name = name;
|
||||
this.docCount = docCount;
|
||||
this.dir = dir;
|
||||
|
@ -124,7 +123,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
this.isCompoundFile = isCompoundFile;
|
||||
this.docStoreOffset = -1;
|
||||
this.docStoreSegment = name;
|
||||
this.segmentCodecs = segmentCodecs;
|
||||
this.codec = codec;
|
||||
delCount = 0;
|
||||
version = Constants.LUCENE_MAIN_VERSION;
|
||||
this.fieldInfos = fieldInfos;
|
||||
|
@ -156,7 +155,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
isCompoundFile = src.isCompoundFile;
|
||||
delCount = src.delCount;
|
||||
segmentCodecs = src.segmentCodecs;
|
||||
codec = src.codec;
|
||||
}
|
||||
|
||||
void setDiagnostics(Map<String, String> diagnostics) {
|
||||
|
@ -177,7 +176,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
* @param format format of the segments info file
|
||||
* @param input input handle to read segment info from
|
||||
*/
|
||||
public SegmentInfo(Directory dir, int format, IndexInput input, CodecProvider codecs) throws IOException {
|
||||
public SegmentInfo(Directory dir, int format, IndexInput input) throws IOException {
|
||||
this.dir = dir;
|
||||
if (format <= DefaultSegmentInfosWriter.FORMAT_3_1) {
|
||||
version = input.readString();
|
||||
|
@ -221,13 +220,13 @@ public final class SegmentInfo implements Cloneable {
|
|||
|
||||
hasProx = input.readByte();
|
||||
|
||||
|
||||
// System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
|
||||
// note: if the codec is not available: Codec.forName will throw an exception.
|
||||
if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
|
||||
segmentCodecs = new SegmentCodecs(codecs, input);
|
||||
codec = Codec.forName(input.readString());
|
||||
} else {
|
||||
// codec ID on FieldInfo is 0 so it will simply use the first codec available
|
||||
// TODO what todo if preflex is not available in the provider? register it or fail?
|
||||
segmentCodecs = new SegmentCodecs(codecs, new Codec[] { codecs.lookup("PreFlex")});
|
||||
codec = Codec.forName("Lucene3x");
|
||||
}
|
||||
diagnostics = input.readStringStringMap();
|
||||
|
||||
|
@ -350,7 +349,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
|
||||
@Override
|
||||
public Object clone() {
|
||||
final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, segmentCodecs,
|
||||
final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, codec,
|
||||
fieldInfos == null ? null : (FieldInfos) fieldInfos.clone());
|
||||
si.docStoreOffset = docStoreOffset;
|
||||
si.docStoreSegment = docStoreSegment;
|
||||
|
@ -573,7 +572,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
output.writeByte((byte) (isCompoundFile ? YES : NO));
|
||||
output.writeInt(delCount);
|
||||
output.writeByte((byte) (hasProx));
|
||||
segmentCodecs.write(output);
|
||||
output.writeString(codec.getName());
|
||||
output.writeStringStringMap(diagnostics);
|
||||
output.writeByte((byte) (hasVectors));
|
||||
}
|
||||
|
@ -583,16 +582,16 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
|
||||
/** Can only be called once. */
|
||||
public void setSegmentCodecs(SegmentCodecs segmentCodecs) {
|
||||
assert this.segmentCodecs == null;
|
||||
if (segmentCodecs == null) {
|
||||
public void setCodec(Codec codec) {
|
||||
assert this.codec == null;
|
||||
if (codec == null) {
|
||||
throw new IllegalArgumentException("segmentCodecs must be non-null");
|
||||
}
|
||||
this.segmentCodecs = segmentCodecs;
|
||||
this.codec = codec;
|
||||
}
|
||||
|
||||
SegmentCodecs getSegmentCodecs() {
|
||||
return segmentCodecs;
|
||||
Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
private void addIfExists(Set<String> files, String fileName) throws IOException {
|
||||
|
@ -628,7 +627,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
for(String ext : IndexFileNames.NON_STORE_INDEX_EXTENSIONS) {
|
||||
addIfExists(fileSet, IndexFileNames.segmentFileName(name, "", ext));
|
||||
}
|
||||
segmentCodecs.files(dir, this, fileSet);
|
||||
codec.files(dir, this, fileSet);
|
||||
}
|
||||
|
||||
if (docStoreOffset != -1) {
|
||||
|
|
|
@ -32,10 +32,11 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosReader;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosWriter;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -83,8 +84,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
// there was an IOException that had interrupted a commit
|
||||
|
||||
public Map<String,String> userData = Collections.<String,String>emptyMap(); // Opaque Map<String, String> that user can specify during IndexWriter.commit
|
||||
|
||||
private CodecProvider codecs;
|
||||
|
||||
private int format;
|
||||
|
||||
|
@ -95,20 +94,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
private transient List<SegmentInfo> cachedUnmodifiableList;
|
||||
private transient Set<SegmentInfo> cachedUnmodifiableSet;
|
||||
|
||||
private Codec codecFormat;
|
||||
|
||||
/**
|
||||
* If non-null, information about loading segments_N files
|
||||
* will be printed here. @see #setInfoStream.
|
||||
*/
|
||||
private static PrintStream infoStream = null;
|
||||
|
||||
public SegmentInfos() {
|
||||
this(CodecProvider.getDefault());
|
||||
}
|
||||
|
||||
public SegmentInfos(CodecProvider codecs) {
|
||||
this.codecs = codecs;
|
||||
}
|
||||
|
||||
public void setFormat(int format) {
|
||||
this.format = format;
|
||||
}
|
||||
|
@ -197,7 +190,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
* since this file might belong to more than one segment (global map) and
|
||||
* could otherwise easily be confused with a per-segment file.
|
||||
*/
|
||||
return IndexFileNames.segmentFileName(""+ version, "", IndexFileNames.GLOBAL_FIELD_NUM_MAP_EXTENSION);
|
||||
return IndexFileNames.segmentFileName("_"+ version, "", IndexFileNames.GLOBAL_FIELD_NUM_MAP_EXTENSION);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -241,9 +234,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public final void read(Directory directory, String segmentFileName,
|
||||
CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
this.codecs = codecs;
|
||||
public final void read(Directory directory, String segmentFileName) throws CorruptIndexException, IOException {
|
||||
boolean success = false;
|
||||
|
||||
// Clear any previous segments:
|
||||
|
@ -253,12 +244,40 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
|
||||
lastGeneration = generation;
|
||||
|
||||
// TODO: scary to have default impl reopen the file... but to make it a bit more flexible,
|
||||
// maybe we could use a plain indexinput here... could default impl rewind/wrap with checksumII,
|
||||
// and any checksumming is then up to implementation?
|
||||
ChecksumIndexInput input = null;
|
||||
try {
|
||||
SegmentInfosReader infosReader = codecs.getSegmentInfosReader();
|
||||
infosReader.read(directory, segmentFileName, codecs, this, IOContext.READ);
|
||||
input = new ChecksumIndexInput(directory.openInput(segmentFileName, IOContext.READ));
|
||||
final int format = input.readInt();
|
||||
setFormat(format);
|
||||
|
||||
// check that it is a format we can understand
|
||||
if (format > DefaultSegmentInfosWriter.FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(segmentFileName, format,
|
||||
DefaultSegmentInfosWriter.FORMAT_MINIMUM, DefaultSegmentInfosWriter.FORMAT_CURRENT);
|
||||
if (format < DefaultSegmentInfosWriter.FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(segmentFileName, format,
|
||||
DefaultSegmentInfosWriter.FORMAT_MINIMUM, DefaultSegmentInfosWriter.FORMAT_CURRENT);
|
||||
|
||||
if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
|
||||
codecFormat = Codec.forName(input.readString());
|
||||
} else {
|
||||
codecFormat = Codec.forName("Lucene3x");
|
||||
}
|
||||
SegmentInfosReader infosReader = codecFormat.segmentInfosFormat().getSegmentInfosReader();
|
||||
infosReader.read(directory, segmentFileName, input, this, IOContext.READ);
|
||||
final long checksumNow = input.getChecksum();
|
||||
final long checksumThen = input.readLong();
|
||||
if (checksumNow != checksumThen)
|
||||
throw new CorruptIndexException("checksum mismatch in segments file");
|
||||
success = true;
|
||||
}
|
||||
finally {
|
||||
if (input != null) {
|
||||
input.close();
|
||||
}
|
||||
if (!success) {
|
||||
// Clear any segment infos we had loaded so we
|
||||
// have a clean slate on retry:
|
||||
|
@ -267,25 +286,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This version of read uses the retry logic (for lock-less
|
||||
* commits) to find the right segments file to load.
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public final void read(Directory directory) throws CorruptIndexException, IOException {
|
||||
read(directory, CodecProvider.getDefault());
|
||||
}
|
||||
|
||||
public final void read(Directory directory, final CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
generation = lastGeneration = -1;
|
||||
this.codecs = codecs;
|
||||
|
||||
new FindSegmentsFile(directory) {
|
||||
|
||||
@Override
|
||||
protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
|
||||
read(directory, segmentFileName, codecs);
|
||||
read(directory, segmentFileName);
|
||||
return null;
|
||||
}
|
||||
}.run();
|
||||
|
@ -297,7 +305,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
// before finishCommit is called
|
||||
IndexOutput pendingSegnOutput;
|
||||
|
||||
private void write(Directory directory) throws IOException {
|
||||
private void write(Directory directory, Codec codec) throws IOException {
|
||||
|
||||
String segmentFileName = getNextSegmentFileName();
|
||||
final String globalFieldMapFile;
|
||||
|
@ -322,8 +330,8 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
boolean success = false;
|
||||
|
||||
try {
|
||||
SegmentInfosWriter infosWriter = codecs.getSegmentInfosWriter();
|
||||
segnOutput = infosWriter.writeInfos(directory, segmentFileName, this, IOContext.DEFAULT);
|
||||
SegmentInfosWriter infosWriter = codec.segmentInfosFormat().getSegmentInfosWriter();
|
||||
segnOutput = infosWriter.writeInfos(directory, segmentFileName, codec.getName(), this, IOContext.DEFAULT);
|
||||
infosWriter.prepareCommit(segnOutput);
|
||||
pendingSegnOutput = segnOutput;
|
||||
success = true;
|
||||
|
@ -380,7 +388,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
sis.cachedUnmodifiableList = null;
|
||||
sis.cachedUnmodifiableSet = null;
|
||||
for(final SegmentInfo info : this) {
|
||||
assert info.getSegmentCodecs() != null;
|
||||
assert info.getCodec() != null;
|
||||
// dont directly access segments, use add method!!!
|
||||
sis.add((SegmentInfo) info.clone());
|
||||
}
|
||||
|
@ -409,7 +417,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static long readCurrentVersion(Directory directory, final CodecProvider codecs)
|
||||
public static long readCurrentVersion(Directory directory)
|
||||
throws CorruptIndexException, IOException {
|
||||
|
||||
// Fully read the segments file: this ensures that it's
|
||||
|
@ -417,8 +425,8 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
// IndexWriter.prepareCommit has been called (but not
|
||||
// yet commit), then the reader will still see itself as
|
||||
// current:
|
||||
SegmentInfos sis = new SegmentInfos(codecs);
|
||||
sis.read(directory, codecs);
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(directory);
|
||||
return sis.version;
|
||||
}
|
||||
|
||||
|
@ -427,10 +435,10 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static Map<String,String> readCurrentUserData(Directory directory, CodecProvider codecs)
|
||||
public static Map<String,String> readCurrentUserData(Directory directory)
|
||||
throws CorruptIndexException, IOException {
|
||||
SegmentInfos sis = new SegmentInfos(codecs);
|
||||
sis.read(directory, codecs);
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(directory);
|
||||
return sis.getUserData();
|
||||
}
|
||||
|
||||
|
@ -808,10 +816,10 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
* method if changes have been made to this {@link SegmentInfos} instance
|
||||
* </p>
|
||||
**/
|
||||
final void prepareCommit(Directory dir) throws IOException {
|
||||
final void prepareCommit(Directory dir, Codec codec) throws IOException {
|
||||
if (pendingSegnOutput != null)
|
||||
throw new IllegalStateException("prepareCommit was already called");
|
||||
write(dir);
|
||||
write(dir, codec);
|
||||
}
|
||||
|
||||
private final long writeGlobalFieldMap(FieldNumberBiMap map, Directory dir, String name) throws IOException {
|
||||
|
@ -882,12 +890,12 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
return files;
|
||||
}
|
||||
|
||||
final void finishCommit(Directory dir) throws IOException {
|
||||
final void finishCommit(Directory dir, Codec codec) throws IOException {
|
||||
if (pendingSegnOutput == null)
|
||||
throw new IllegalStateException("prepareCommit was not called");
|
||||
boolean success = false;
|
||||
try {
|
||||
SegmentInfosWriter infosWriter = codecs.getSegmentInfosWriter();
|
||||
SegmentInfosWriter infosWriter = codec.segmentInfosFormat().getSegmentInfosWriter();
|
||||
infosWriter.finishCommit(pendingSegnOutput);
|
||||
pendingSegnOutput = null;
|
||||
success = true;
|
||||
|
@ -958,9 +966,9 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
* method if changes have been made to this {@link SegmentInfos} instance
|
||||
* </p>
|
||||
**/
|
||||
final void commit(Directory dir) throws IOException {
|
||||
prepareCommit(dir);
|
||||
finishCommit(dir);
|
||||
final void commit(Directory dir, Codec codec) throws IOException {
|
||||
prepareCommit(dir, codec);
|
||||
finishCommit(dir, codec);
|
||||
}
|
||||
|
||||
public String toString(Directory directory) {
|
||||
|
@ -1106,7 +1114,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
if (cloneChildren) {
|
||||
final List<SegmentInfo> list = new ArrayList<SegmentInfo>(size());
|
||||
for(final SegmentInfo info : this) {
|
||||
assert info.getSegmentCodecs() != null;
|
||||
assert info.getCodec() != null;
|
||||
list.add((SegmentInfo) info.clone());
|
||||
}
|
||||
return list;
|
||||
|
@ -1120,6 +1128,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
this.addAll(infos);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the codec used to decode this SegmentInfos from disk
|
||||
* @lucene.internal
|
||||
*/
|
||||
Codec codecFormat() {
|
||||
return codecFormat;
|
||||
}
|
||||
|
||||
/** Returns an <b>unmodifiable</b> {@link Iterator} of contained segments in order. */
|
||||
// @Override (comment out until Java 6)
|
||||
public Iterator<SegmentInfo> iterator() {
|
||||
|
|
|
@ -73,7 +73,7 @@ final class SegmentMerger {
|
|||
|
||||
private IOContext context;
|
||||
|
||||
SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, IOContext context) {
|
||||
SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, Codec codec, IOContext context) {
|
||||
this.payloadProcessorProvider = payloadProcessorProvider;
|
||||
directory = dir;
|
||||
segment = name;
|
||||
|
@ -89,6 +89,7 @@ final class SegmentMerger {
|
|||
};
|
||||
}
|
||||
this.termIndexInterval = termIndexInterval;
|
||||
this.codec = codec;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
|
@ -254,12 +255,11 @@ final class SegmentMerger {
|
|||
fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.DOC_VALUES), false);
|
||||
}
|
||||
}
|
||||
final SegmentCodecs codecInfo = fieldInfos.buildSegmentCodecs(false);
|
||||
|
||||
int docCount = 0;
|
||||
|
||||
setMatchingSegmentReaders();
|
||||
final FieldsWriter fieldsWriter = codecInfo.provider.fieldsWriter(directory, segment, context);
|
||||
final FieldsWriter fieldsWriter = codec.fieldsFormat().fieldsWriter(directory, segment, context);
|
||||
try {
|
||||
int idx = 0;
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : readers) {
|
||||
|
@ -293,7 +293,7 @@ final class SegmentMerger {
|
|||
// entering the index. See LUCENE-1282 for
|
||||
// details.
|
||||
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
|
||||
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null, context);
|
||||
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codec, null, context);
|
||||
|
||||
return docCount;
|
||||
}
|
||||
|
@ -493,9 +493,9 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
|
||||
SegmentCodecs getSegmentCodecs() {
|
||||
Codec getCodec() {
|
||||
assert segmentWriteState != null;
|
||||
return segmentWriteState.segmentCodecs;
|
||||
return segmentWriteState.codec;
|
||||
}
|
||||
|
||||
private final void mergeTerms() throws CorruptIndexException, IOException {
|
||||
|
@ -566,8 +566,8 @@ final class SegmentMerger {
|
|||
mergeState.dirPayloadProcessor[i] = payloadProcessorProvider.getDirProcessor(reader.reader.directory());
|
||||
}
|
||||
}
|
||||
codec = segmentWriteState.segmentCodecs.codec();
|
||||
final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState);
|
||||
|
||||
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
|
||||
boolean success = false;
|
||||
try {
|
||||
consumer.merge(mergeState,
|
||||
|
@ -584,7 +584,7 @@ final class SegmentMerger {
|
|||
}
|
||||
|
||||
private void mergePerDoc() throws IOException {
|
||||
final PerDocConsumer docsConsumer = codec
|
||||
final PerDocConsumer docsConsumer = codec.docValuesFormat()
|
||||
.docsConsumer(new PerDocWriteState(segmentWriteState));
|
||||
// TODO: remove this check when 3.x indexes are no longer supported
|
||||
// (3.x indexes don't have docvalues)
|
||||
|
|
|
@ -35,11 +35,11 @@ public class SegmentReadState {
|
|||
// that must do so), then it should negate this value to
|
||||
// get the app's terms divisor:
|
||||
public int termsIndexDivisor;
|
||||
public final int codecId;
|
||||
public final String segmentSuffix;
|
||||
|
||||
public SegmentReadState(Directory dir, SegmentInfo info,
|
||||
FieldInfos fieldInfos, IOContext context, int termsIndexDivisor) {
|
||||
this(dir, info, fieldInfos, context, termsIndexDivisor, -1);
|
||||
this(dir, info, fieldInfos, context, termsIndexDivisor, "");
|
||||
}
|
||||
|
||||
public SegmentReadState(Directory dir,
|
||||
|
@ -47,12 +47,22 @@ public class SegmentReadState {
|
|||
FieldInfos fieldInfos,
|
||||
IOContext context,
|
||||
int termsIndexDivisor,
|
||||
int codecId) {
|
||||
String segmentSuffix) {
|
||||
this.dir = dir;
|
||||
this.segmentInfo = info;
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.context = context;
|
||||
this.termsIndexDivisor = termsIndexDivisor;
|
||||
this.codecId = codecId;
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
}
|
||||
}
|
||||
|
||||
public SegmentReadState(SegmentReadState other,
|
||||
String newSegmentSuffix) {
|
||||
this.dir = other.dir;
|
||||
this.segmentInfo = other.segmentInfo;
|
||||
this.fieldInfos = other.fieldInfos;
|
||||
this.context = other.context;
|
||||
this.termsIndexDivisor = other.termsIndexDivisor;
|
||||
this.segmentSuffix = newSegmentSuffix;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.PrintStream;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
|
@ -43,8 +44,8 @@ public class SegmentWriteState {
|
|||
// Lazily created:
|
||||
public BitVector liveDocs;
|
||||
|
||||
final SegmentCodecs segmentCodecs;
|
||||
public final int codecId;
|
||||
public final Codec codec;
|
||||
public final String segmentSuffix;
|
||||
|
||||
/** Expert: The fraction of terms in the "dictionary" which should be stored
|
||||
* in RAM. Smaller values use more memory, but make searching slightly
|
||||
|
@ -56,7 +57,7 @@ public class SegmentWriteState {
|
|||
public final IOContext context;
|
||||
|
||||
public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
|
||||
int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, BufferedDeletes segDeletes, IOContext context) {
|
||||
int numDocs, int termIndexInterval, Codec codec, BufferedDeletes segDeletes, IOContext context) {
|
||||
this.infoStream = infoStream;
|
||||
this.segDeletes = segDeletes;
|
||||
this.directory = directory;
|
||||
|
@ -64,24 +65,24 @@ public class SegmentWriteState {
|
|||
this.fieldInfos = fieldInfos;
|
||||
this.numDocs = numDocs;
|
||||
this.termIndexInterval = termIndexInterval;
|
||||
this.segmentCodecs = segmentCodecs;
|
||||
codecId = -1;
|
||||
this.codec = codec;
|
||||
segmentSuffix = "";
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a shallow {@link SegmentWriteState} copy final a codec ID
|
||||
* Create a shallow {@link SegmentWriteState} copy final a format ID
|
||||
*/
|
||||
SegmentWriteState(SegmentWriteState state, int codecId) {
|
||||
public SegmentWriteState(SegmentWriteState state, String segmentSuffix) {
|
||||
infoStream = state.infoStream;
|
||||
directory = state.directory;
|
||||
segmentName = state.segmentName;
|
||||
fieldInfos = state.fieldInfos;
|
||||
numDocs = state.numDocs;
|
||||
termIndexInterval = state.termIndexInterval;
|
||||
segmentCodecs = state.segmentCodecs;
|
||||
context = state.context;
|
||||
this.codecId = codecId;
|
||||
codec = state.codec;
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
segDeletes = state.segDeletes;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsWriter;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
|
@ -35,12 +35,12 @@ final class StoredFieldsWriter {
|
|||
int freeCount;
|
||||
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final CodecProvider codecProvider;
|
||||
final Codec codec;
|
||||
|
||||
public StoredFieldsWriter(DocumentsWriterPerThread docWriter) {
|
||||
this.docWriter = docWriter;
|
||||
this.docState = docWriter.docState;
|
||||
this.codecProvider = docWriter.codecProvider;
|
||||
this.codec = docWriter.codec;
|
||||
}
|
||||
|
||||
private int numStoredFields;
|
||||
|
@ -81,7 +81,7 @@ final class StoredFieldsWriter {
|
|||
|
||||
private synchronized void initFieldsWriter(IOContext context) throws IOException {
|
||||
if (fieldsWriter == null) {
|
||||
fieldsWriter = codecProvider.fieldsWriter(docWriter.directory, docWriter.getSegment(), context);
|
||||
fieldsWriter = codec.fieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegment(), context);
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -109,14 +109,14 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
// private String segment;
|
||||
|
||||
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, IOContext context,
|
||||
int termsCacheSize, int codecId)
|
||||
int termsCacheSize, String segmentSuffix)
|
||||
throws IOException {
|
||||
|
||||
this.postingsReader = postingsReader;
|
||||
termsCache = new DoubleBarrelLRUCache<FieldAndTerm,BlockTermState>(termsCacheSize);
|
||||
|
||||
// this.segment = segment;
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTermsWriter.TERMS_EXTENSION),
|
||||
context);
|
||||
|
||||
boolean success = false;
|
||||
|
@ -194,12 +194,8 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, BlockTermsWriter.TERMS_EXTENSION));
|
||||
}
|
||||
|
||||
public static void getExtensions(Collection<String> extensions) {
|
||||
extensions.add(BlockTermsWriter.TERMS_EXTENSION);
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockTermsWriter.TERMS_EXTENSION));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -71,7 +71,7 @@ public class BlockTermsWriter extends FieldsConsumer {
|
|||
public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
|
||||
SegmentWriteState state, PostingsWriterBase postingsWriter)
|
||||
throws IOException {
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_EXTENSION);
|
||||
this.termsIndexWriter = termsIndexWriter;
|
||||
out = state.directory.createOutput(termsFileName, state.context);
|
||||
boolean success = false;
|
||||
|
|
|
@ -110,13 +110,13 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
|
||||
public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, String segment,
|
||||
PostingsReaderBase postingsReader, IOContext ioContext,
|
||||
int codecId, int indexDivisor)
|
||||
String segmentSuffix, int indexDivisor)
|
||||
throws IOException {
|
||||
|
||||
this.postingsReader = postingsReader;
|
||||
|
||||
this.segment = segment;
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTreeTermsWriter.TERMS_EXTENSION),
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION),
|
||||
ioContext);
|
||||
|
||||
boolean success = false;
|
||||
|
@ -125,7 +125,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
try {
|
||||
readHeader(in);
|
||||
if (indexDivisor != -1) {
|
||||
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
|
||||
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
|
||||
ioContext);
|
||||
readIndexHeader(indexIn);
|
||||
}
|
||||
|
@ -206,14 +206,9 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, BlockTreeTermsWriter.TERMS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
public static void getExtensions(Collection<String> extensions) {
|
||||
extensions.add(BlockTreeTermsWriter.TERMS_EXTENSION);
|
||||
extensions.add(BlockTreeTermsWriter.TERMS_INDEX_EXTENSION);
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -145,7 +145,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
|
|||
throw new IllegalArgumentException("maxItemsInBlock must be at least 2*(minItemsInBlock-1); got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock);
|
||||
}
|
||||
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_EXTENSION);
|
||||
out = state.directory.createOutput(termsFileName, state.context);
|
||||
boolean success = false;
|
||||
IndexOutput indexOut = null;
|
||||
|
@ -157,7 +157,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
|
|||
|
||||
//DEBUG = state.segmentName.equals("_4a");
|
||||
|
||||
final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
|
||||
final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_INDEX_EXTENSION);
|
||||
indexOut = state.directory.createOutput(termsIndexFileName, state.context);
|
||||
writeIndexHeader(indexOut);
|
||||
|
||||
|
|
|
@ -20,59 +20,67 @@ package org.apache.lucene.index.codecs;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.util.NamedSPILoader;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public abstract class Codec {
|
||||
public static final Codec[] EMPTY = new Codec[0];
|
||||
/** Unique name that's used to retrieve this codec when
|
||||
* reading the index */
|
||||
public final String name;
|
||||
|
||||
protected Codec(String name) {
|
||||
/**
|
||||
* Encodes/decodes an inverted index segment
|
||||
*/
|
||||
public abstract class Codec implements NamedSPILoader.NamedSPI {
|
||||
|
||||
private static final NamedSPILoader<Codec> loader =
|
||||
new NamedSPILoader<Codec>(Codec.class);
|
||||
|
||||
private final String name;
|
||||
|
||||
public Codec(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
/** Writes a new segment */
|
||||
public abstract FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException;
|
||||
|
||||
public static void debug(String s, String desc) {
|
||||
if (desc != null) {
|
||||
System.out.println(Thread.currentThread().getName()+ " [" + desc + "]:" + s);
|
||||
} else {
|
||||
System.out.println(Thread.currentThread().getName() + ": " + s);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
public static void debug(String s) {
|
||||
debug(s, null);
|
||||
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
postingsFormat().files(dir, info, "", files);
|
||||
//TODO: not yet fieldsFormat().files(dir, info, files);
|
||||
docValuesFormat().files(dir, info, files);
|
||||
}
|
||||
|
||||
/** Encodes/decodes postings */
|
||||
public abstract PostingsFormat postingsFormat();
|
||||
|
||||
/** Encodes/decodes docvalues */
|
||||
public abstract DocValuesFormat docValuesFormat();
|
||||
|
||||
/** Encodes/decodes stored fields, term vectors, fieldinfos */
|
||||
public abstract FieldsFormat fieldsFormat();
|
||||
|
||||
public abstract SegmentInfosFormat segmentInfosFormat();
|
||||
|
||||
/** looks up a codec by name */
|
||||
public static Codec forName(String name) {
|
||||
return loader.lookup(name);
|
||||
}
|
||||
|
||||
/** returns a list of all available codec names */
|
||||
public static Set<String> availableCodecs() {
|
||||
return loader.availableServices();
|
||||
}
|
||||
|
||||
private static Codec defaultCodec = Codec.forName("Lucene40");
|
||||
|
||||
// TODO: should we use this, or maybe a system property is better?
|
||||
public static Codec getDefault() {
|
||||
return defaultCodec;
|
||||
}
|
||||
|
||||
public static void setDefault(Codec codec) {
|
||||
defaultCodec = codec;
|
||||
}
|
||||
|
||||
/** Reads a segment. NOTE: by the time this call
|
||||
* returns, it must hold open any files it will need to
|
||||
* use; else, those files may be deleted. */
|
||||
public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
|
||||
|
||||
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
|
||||
|
||||
public abstract PerDocValues docsProducer(SegmentReadState state) throws IOException;
|
||||
|
||||
/**
|
||||
* Gathers files associated with this segment
|
||||
*
|
||||
* @param dir the {@link Directory} this segment was written to
|
||||
* @param segmentInfo the {@link SegmentInfo} for this segment
|
||||
* @param id the codec id within this segment
|
||||
* @param files the of files to add the codec files to.
|
||||
*/
|
||||
public abstract void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException;
|
||||
|
||||
/** Records all file extensions this codec uses */
|
||||
public abstract void getExtensions(Set<String> extensions);
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return name;
|
||||
|
|
|
@ -1,199 +0,0 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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 java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/** Holds a set of codecs, keyed by name. You subclass
|
||||
* this, instantiate it, and register your codecs, then
|
||||
* pass this instance to IndexReader/IndexWriter (via
|
||||
* package private APIs) to use different codecs when
|
||||
* reading & writing segments.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
public class CodecProvider {
|
||||
private SegmentInfosWriter infosWriter = new DefaultSegmentInfosWriter();
|
||||
private SegmentInfosReader infosReader = new DefaultSegmentInfosReader();
|
||||
private String defaultFieldCodec = "Standard";
|
||||
private final Map<String, String> perFieldMap = new HashMap<String, String>();
|
||||
|
||||
|
||||
private final HashMap<String, Codec> codecs = new HashMap<String, Codec>();
|
||||
|
||||
private final Set<String> knownExtensions = new HashSet<String>();
|
||||
|
||||
|
||||
public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText", "Memory"};
|
||||
|
||||
public synchronized void register(Codec codec) {
|
||||
if (codec.name == null) {
|
||||
throw new IllegalArgumentException("code.name is null");
|
||||
}
|
||||
if (!codecs.containsKey(codec.name)) {
|
||||
codecs.put(codec.name, codec);
|
||||
codec.getExtensions(knownExtensions);
|
||||
} else if (codecs.get(codec.name) != codec) {
|
||||
throw new IllegalArgumentException("codec '" + codec.name + "' is already registered as a different codec instance");
|
||||
}
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public synchronized void unregister(Codec codec) {
|
||||
if (codec.name == null) {
|
||||
throw new IllegalArgumentException("code.name is null");
|
||||
}
|
||||
if (codecs.containsKey(codec.name)) {
|
||||
Codec c = codecs.get(codec.name);
|
||||
if (codec == c) {
|
||||
codecs.remove(codec.name);
|
||||
} else {
|
||||
throw new IllegalArgumentException("codec '" + codec.name + "' is being impersonated by a different codec instance!!!");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public synchronized Set<String> listAll() {
|
||||
return codecs.keySet();
|
||||
}
|
||||
|
||||
public Collection<String> getAllExtensions() {
|
||||
return knownExtensions;
|
||||
}
|
||||
|
||||
public synchronized Codec lookup(String name) {
|
||||
final Codec codec = codecs.get(name);
|
||||
if (codec == null) {
|
||||
throw new IllegalArgumentException("required codec '" + name + "' not found; known codecs: " + codecs.keySet());
|
||||
}
|
||||
return codec;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff a codec with the given name is registered
|
||||
* @param name codec name
|
||||
* @return <code>true</code> iff a codec with the given name is registered, otherwise <code>false</code>.
|
||||
*/
|
||||
public synchronized boolean isCodecRegistered(String name) {
|
||||
return codecs.containsKey(name);
|
||||
}
|
||||
|
||||
public SegmentInfosWriter getSegmentInfosWriter() {
|
||||
return infosWriter;
|
||||
}
|
||||
|
||||
public SegmentInfosReader getSegmentInfosReader() {
|
||||
return infosReader;
|
||||
}
|
||||
|
||||
/** expert */
|
||||
public FieldsReader fieldsReader(Directory directory, String segment, FieldInfos fn, IOContext context, int docStoreOffset, int size) throws IOException {
|
||||
return new DefaultFieldsReader(directory, segment, fn, context, docStoreOffset, size);
|
||||
}
|
||||
|
||||
/** expert */
|
||||
public FieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
return new DefaultFieldsWriter(directory, segment, context);
|
||||
}
|
||||
|
||||
static private CodecProvider defaultCodecs = new CoreCodecProvider();
|
||||
|
||||
public static CodecProvider getDefault() {
|
||||
return defaultCodecs;
|
||||
}
|
||||
|
||||
/** For testing only
|
||||
* @lucene.internal */
|
||||
public static void setDefault(CodecProvider cp) {
|
||||
defaultCodecs = cp;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link Codec} for a given field. Not that setting a field's codec is
|
||||
* write-once. If the field's codec is already set this method will throw an
|
||||
* {@link IllegalArgumentException}.
|
||||
*
|
||||
* @param field
|
||||
* the name of the field
|
||||
* @param codec
|
||||
* the name of the codec
|
||||
* @throws IllegalArgumentException
|
||||
* if the codec for the given field is already set
|
||||
*
|
||||
*/
|
||||
public synchronized void setFieldCodec(String field, String codec) {
|
||||
if (perFieldMap.containsKey(field))
|
||||
throw new IllegalArgumentException("codec for field: " + field
|
||||
+ " already set to " + perFieldMap.get(field));
|
||||
perFieldMap.put(field, codec);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link Codec} name for the given field or the default codec if
|
||||
* not set.
|
||||
*
|
||||
* @param name
|
||||
* the fields name
|
||||
* @return the {@link Codec} name for the given field or the default codec if
|
||||
* not set.
|
||||
*/
|
||||
public synchronized String getFieldCodec(String name) {
|
||||
final String codec;
|
||||
if ((codec = perFieldMap.get(name)) == null) {
|
||||
return defaultFieldCodec;
|
||||
}
|
||||
return codec;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if this provider has a Codec registered for this
|
||||
* field.
|
||||
*/
|
||||
public synchronized boolean hasFieldCodec(String name) {
|
||||
return perFieldMap.containsKey(name);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the default {@link Codec} for this {@link CodecProvider}
|
||||
*
|
||||
* @return the default {@link Codec} for this {@link CodecProvider}
|
||||
*/
|
||||
public synchronized String getDefaultFieldCodec() {
|
||||
return defaultFieldCodec;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the default {@link Codec} for this {@link CodecProvider}
|
||||
*
|
||||
* @param codec
|
||||
* the codecs name
|
||||
*/
|
||||
public synchronized void setDefaultFieldCodec(String codec) {
|
||||
defaultFieldCodec = codec;
|
||||
}
|
||||
}
|
|
@ -1,51 +0,0 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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.index.codecs.memory.MemoryCodec;
|
||||
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
|
||||
import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
|
||||
import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
|
||||
/**
|
||||
* A CodecProvider that registers all core codecs that ship
|
||||
* with Lucene. This will not register any user codecs, but
|
||||
* you can easily instantiate this class and register them
|
||||
* yourself and specify per-field codecs:
|
||||
*
|
||||
* <pre>
|
||||
* CodecProvider cp = new CoreCodecProvider();
|
||||
* cp.register(new MyFastCodec());
|
||||
* cp.setDefaultFieldCodec("Standard");
|
||||
* cp.setFieldCodec("id", "Pulsing");
|
||||
* cp.setFieldCodec("body", "MyFastCodec");
|
||||
* IndexWriterConfig iwc = new IndexWriterConfig(analyzer);
|
||||
* iwc.setCodecProvider(cp);
|
||||
* </pre>
|
||||
*/
|
||||
|
||||
public class CoreCodecProvider extends CodecProvider {
|
||||
public CoreCodecProvider() {
|
||||
register(new StandardCodec());
|
||||
register(new PreFlexCodec());
|
||||
register(new PulsingCodec());
|
||||
register(new SimpleTextCodec());
|
||||
register(new MemoryCodec());
|
||||
}
|
||||
}
|
|
@ -25,7 +25,6 @@ import org.apache.lucene.index.FieldInfos;
|
|||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.codecs.DocValuesWriterBase;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
|
@ -34,42 +33,45 @@ import org.apache.lucene.store.Directory;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public class DefaultDocValuesConsumer extends DocValuesWriterBase {
|
||||
private final Directory directory;
|
||||
private final Directory mainDirectory;
|
||||
private Directory directory;
|
||||
|
||||
final static String DOC_VALUES_SEGMENT_SUFFIX = "dv";
|
||||
|
||||
public DefaultDocValuesConsumer(PerDocWriteState state) throws IOException {
|
||||
super(state);
|
||||
mainDirectory = state.directory;
|
||||
//TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
|
||||
this.directory = new CompoundFileDirectory(state.directory,
|
||||
IndexFileNames.segmentFileName(state.segmentName, state.codecId,
|
||||
IndexFileNames.COMPOUND_FILE_EXTENSION), state.context, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Directory getDirectory() {
|
||||
protected Directory getDirectory() throws IOException {
|
||||
// lazy init
|
||||
if (directory == null) {
|
||||
directory = new CompoundFileDirectory(mainDirectory,
|
||||
IndexFileNames.segmentFileName(segmentName, DOC_VALUES_SEGMENT_SUFFIX,
|
||||
IndexFileNames.COMPOUND_FILE_EXTENSION), context, true);
|
||||
}
|
||||
return directory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
this.directory.close();
|
||||
if (directory != null) {
|
||||
directory.close();
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough")
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.getCodecId() == codecId && fieldInfo.hasDocValues()) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
return;
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void getExtensions(Set<String> extensions) {
|
||||
extensions.add(IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
|
||||
extensions.add(IndexFileNames.COMPOUND_FILE_EXTENSION);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,44 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
public class DefaultDocValuesFormat extends DocValuesFormat {
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new DefaultDocValuesConsumer(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new DefaultDocValuesProducer(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
DefaultDocValuesConsumer.files(dir, info, files);
|
||||
}
|
||||
}
|
|
@ -26,7 +26,6 @@ import java.util.TreeMap;
|
|||
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.DocValuesReaderBase;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -37,7 +36,7 @@ import org.apache.lucene.util.IOUtils;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public class DefaultDocValuesProducer extends DocValuesReaderBase {
|
||||
protected final TreeMap<String, IndexDocValues> docValues;
|
||||
protected final TreeMap<String,IndexDocValues> docValues;
|
||||
private final Directory cfs;
|
||||
|
||||
/**
|
||||
|
@ -45,10 +44,16 @@ public class DefaultDocValuesProducer extends DocValuesReaderBase {
|
|||
* {@link IndexDocValues} instances for this segment and codec.
|
||||
*/
|
||||
public DefaultDocValuesProducer(SegmentReadState state) throws IOException {
|
||||
cfs = new CompoundFileDirectory(state.dir,
|
||||
IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, IndexFileNames.COMPOUND_FILE_EXTENSION),
|
||||
state.context, false);
|
||||
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, cfs, state.codecId, state.context);
|
||||
if (state.fieldInfos.anyDocValuesFields()) {
|
||||
cfs = new CompoundFileDirectory(state.dir,
|
||||
IndexFileNames.segmentFileName(state.segmentInfo.name,
|
||||
DefaultDocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION),
|
||||
state.context, false);
|
||||
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, cfs, state.context);
|
||||
} else {
|
||||
cfs = null;
|
||||
docValues = new TreeMap<String,IndexDocValues>();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -58,8 +63,12 @@ public class DefaultDocValuesProducer extends DocValuesReaderBase {
|
|||
|
||||
@Override
|
||||
protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
|
||||
final ArrayList<Closeable> list = new ArrayList<Closeable>(closeables);
|
||||
list.add(cfs);
|
||||
IOUtils.close(list);
|
||||
if (cfs != null) {
|
||||
final ArrayList<Closeable> list = new ArrayList<Closeable>(closeables);
|
||||
list.add(cfs);
|
||||
IOUtils.close(list);
|
||||
} else {
|
||||
IOUtils.close(closeables);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,48 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public class DefaultFieldsFormat extends FieldsFormat {
|
||||
|
||||
@Override
|
||||
public FieldsReader fieldsReader(Directory directory, String segment,
|
||||
FieldInfos fn, IOContext context, int docStoreOffset, int size)
|
||||
throws IOException {
|
||||
return new DefaultFieldsReader(directory, segment, fn, context, docStoreOffset, size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsWriter fieldsWriter(Directory directory, String segment,
|
||||
IOContext context) throws IOException {
|
||||
return new DefaultFieldsWriter(directory, segment, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
// TODO!
|
||||
}
|
||||
}
|
|
@ -82,7 +82,7 @@ public final class DefaultFieldsReader extends FieldsReader implements Cloneable
|
|||
|
||||
/** Verifies that the code version which wrote the segment is supported. */
|
||||
public static void checkCodeVersion(Directory dir, String segment) throws IOException {
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", DefaultFieldsWriter.FIELDS_INDEX_EXTENSION);
|
||||
IndexInput idxStream = dir.openInput(indexStreamFN, IOContext.DEFAULT);
|
||||
|
||||
try {
|
||||
|
@ -121,8 +121,8 @@ public final class DefaultFieldsReader extends FieldsReader implements Cloneable
|
|||
try {
|
||||
fieldInfos = fn;
|
||||
|
||||
cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), context);
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
|
||||
cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", DefaultFieldsWriter.FIELDS_EXTENSION), context);
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", DefaultFieldsWriter.FIELDS_INDEX_EXTENSION);
|
||||
cloneableIndexStream = d.openInput(indexStreamFN, context);
|
||||
|
||||
format = cloneableIndexStream.readInt();
|
||||
|
|
|
@ -61,6 +61,10 @@ public final class DefaultFieldsWriter extends FieldsWriter {
|
|||
// when removing support for old versions, leave the last supported version here
|
||||
static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
|
||||
|
||||
// TODO: remove from IndexFileNames
|
||||
public static final String FIELDS_EXTENSION = IndexFileNames.FIELDS_EXTENSION;
|
||||
public static final String FIELDS_INDEX_EXTENSION = IndexFileNames.FIELDS_INDEX_EXTENSION;
|
||||
|
||||
// If null - we were supplied with streams, if notnull - we manage them ourselves
|
||||
private Directory directory;
|
||||
private String segment;
|
||||
|
@ -73,8 +77,8 @@ public final class DefaultFieldsWriter extends FieldsWriter {
|
|||
|
||||
boolean success = false;
|
||||
try {
|
||||
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), context);
|
||||
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION), context);
|
||||
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
|
||||
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION), context);
|
||||
|
||||
fieldsStream.writeInt(FORMAT_CURRENT);
|
||||
indexStream.writeInt(FORMAT_CURRENT);
|
||||
|
@ -129,11 +133,11 @@ public final class DefaultFieldsWriter extends FieldsWriter {
|
|||
} catch (IOException ignored) {
|
||||
}
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION));
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,36 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class DefaultSegmentInfosFormat extends SegmentInfosFormat {
|
||||
private final SegmentInfosReader reader = new DefaultSegmentInfosReader();
|
||||
private final SegmentInfosWriter writer = new DefaultSegmentInfosWriter();
|
||||
|
||||
@Override
|
||||
public SegmentInfosReader getSegmentInfosReader() {
|
||||
return reader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentInfosWriter getSegmentInfosWriter() {
|
||||
return writer;
|
||||
}
|
||||
}
|
|
@ -19,17 +19,14 @@ package org.apache.lucene.index.codecs;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||
import org.apache.lucene.index.IndexFormatTooNewException;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentInfos;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link SegmentInfosReader}.
|
||||
|
@ -37,90 +34,60 @@ import org.apache.lucene.store.IndexInput;
|
|||
*/
|
||||
public class DefaultSegmentInfosReader extends SegmentInfosReader {
|
||||
|
||||
// TODO: shove all backwards code to preflex!
|
||||
// this is a little tricky, because of IR.commit(), two options:
|
||||
// 1. PreFlex writes 4.x SIS format, but reads both 3.x and 4.x
|
||||
// (and maybe RW always only writes the 3.x one? for that to work well,
|
||||
// we have to move .fnx file to codec too, not too bad but more work).
|
||||
// or we just have crappier RW testing like today.
|
||||
// 2. PreFlex writes 3.x SIS format, and only reads 3.x
|
||||
// (in this case we have to move .fnx file to codec as well)
|
||||
@Override
|
||||
public void read(Directory directory, String segmentsFileName, CodecProvider codecs,
|
||||
SegmentInfos infos, IOContext context) throws IOException {
|
||||
IndexInput input = null;
|
||||
try {
|
||||
input = openInput(directory, segmentsFileName, context);
|
||||
final int format = input.readInt();
|
||||
infos.setFormat(format);
|
||||
|
||||
// check that it is a format we can understand
|
||||
if (format > DefaultSegmentInfosWriter.FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(segmentsFileName, format,
|
||||
DefaultSegmentInfosWriter.FORMAT_MINIMUM, DefaultSegmentInfosWriter.FORMAT_CURRENT);
|
||||
if (format < DefaultSegmentInfosWriter.FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(segmentsFileName, format,
|
||||
DefaultSegmentInfosWriter.FORMAT_MINIMUM, DefaultSegmentInfosWriter.FORMAT_CURRENT);
|
||||
|
||||
infos.version = input.readLong(); // read version
|
||||
infos.counter = input.readInt(); // read counter
|
||||
if (infos.getFormat() <= DefaultSegmentInfosWriter.FORMAT_4_0) {
|
||||
infos.setGlobalFieldMapVersion(input.readLong());
|
||||
}
|
||||
for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
|
||||
SegmentInfo si = new SegmentInfo(directory, format, input, codecs);
|
||||
if (si.getVersion() == null) {
|
||||
// Could be a 3.0 - try to open the doc stores - if it fails, it's a
|
||||
// 2.x segment, and an IndexFormatTooOldException will be thrown,
|
||||
// which is what we want.
|
||||
Directory dir = directory;
|
||||
if (si.getDocStoreOffset() != -1) {
|
||||
if (si.getDocStoreIsCompoundFile()) {
|
||||
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
|
||||
si.getDocStoreSegment(), "",
|
||||
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context, false);
|
||||
}
|
||||
} else if (si.getUseCompoundFile()) {
|
||||
dir = new CompoundFileDirectory(dir,IndexFileNames.segmentFileName(
|
||||
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
|
||||
}
|
||||
|
||||
try {
|
||||
DefaultFieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
|
||||
} finally {
|
||||
// If we opened the directory, close it
|
||||
if (dir != directory) dir.close();
|
||||
}
|
||||
|
||||
// Above call succeeded, so it's a 3.0 segment. Upgrade it so the next
|
||||
// time the segment is read, its version won't be null and we won't
|
||||
// need to open FieldsReader every time for each such segment.
|
||||
si.setVersion("3.0");
|
||||
} else if (si.getVersion().equals("2.x")) {
|
||||
// If it's a 3x index touched by 3.1+ code, then segments record their
|
||||
// version, whether they are 2.x ones or not. We detect that and throw
|
||||
// appropriate exception.
|
||||
throw new IndexFormatTooOldException(si.name, si.getVersion());
|
||||
}
|
||||
infos.add(si);
|
||||
}
|
||||
|
||||
infos.userData = input.readStringStringMap();
|
||||
finalizeInput(input);
|
||||
|
||||
} finally {
|
||||
if (input != null) {
|
||||
input.close();
|
||||
}
|
||||
public void read(Directory directory, String segmentsFileName, ChecksumIndexInput input, SegmentInfos infos, IOContext context) throws IOException {
|
||||
infos.version = input.readLong(); // read version
|
||||
infos.counter = input.readInt(); // read counter
|
||||
final int format = infos.getFormat();
|
||||
if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
|
||||
infos.setGlobalFieldMapVersion(input.readLong());
|
||||
}
|
||||
for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
|
||||
SegmentInfo si = new SegmentInfo(directory, format, input);
|
||||
if (si.getVersion() == null) {
|
||||
// Could be a 3.0 - try to open the doc stores - if it fails, it's a
|
||||
// 2.x segment, and an IndexFormatTooOldException will be thrown,
|
||||
// which is what we want.
|
||||
Directory dir = directory;
|
||||
if (si.getDocStoreOffset() != -1) {
|
||||
if (si.getDocStoreIsCompoundFile()) {
|
||||
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
|
||||
si.getDocStoreSegment(), "",
|
||||
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context, false);
|
||||
}
|
||||
} else if (si.getUseCompoundFile()) {
|
||||
dir = new CompoundFileDirectory(dir,IndexFileNames.segmentFileName(
|
||||
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
|
||||
}
|
||||
|
||||
try {
|
||||
DefaultFieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
|
||||
} finally {
|
||||
// If we opened the directory, close it
|
||||
if (dir != directory) dir.close();
|
||||
}
|
||||
|
||||
// Above call succeeded, so it's a 3.0 segment. Upgrade it so the next
|
||||
// time the segment is read, its version won't be null and we won't
|
||||
// need to open FieldsReader every time for each such segment.
|
||||
si.setVersion("3.0");
|
||||
} else if (si.getVersion().equals("2.x")) {
|
||||
// If it's a 3x index touched by 3.1+ code, then segments record their
|
||||
// version, whether they are 2.x ones or not. We detect that and throw
|
||||
// appropriate exception.
|
||||
throw new IndexFormatTooOldException(si.name, si.getVersion());
|
||||
}
|
||||
infos.add(si);
|
||||
}
|
||||
|
||||
infos.userData = input.readStringStringMap();
|
||||
}
|
||||
|
||||
public IndexInput openInput(Directory dir, String segmentsFileName, IOContext context) throws IOException {
|
||||
IndexInput in = dir.openInput(segmentsFileName, context);
|
||||
return new ChecksumIndexInput(in);
|
||||
|
||||
}
|
||||
|
||||
public void finalizeInput(IndexInput input) throws IOException, CorruptIndexException {
|
||||
ChecksumIndexInput cksumInput = (ChecksumIndexInput)input;
|
||||
final long checksumNow = cksumInput.getChecksum();
|
||||
final long checksumThen = cksumInput.readLong();
|
||||
if (checksumNow != checksumThen)
|
||||
throw new CorruptIndexException("checksum mismatch in segments file");
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -56,12 +56,13 @@ public class DefaultSegmentInfosWriter extends SegmentInfosWriter {
|
|||
public static final int FORMAT_MINIMUM = FORMAT_DIAGNOSTICS;
|
||||
|
||||
@Override
|
||||
public IndexOutput writeInfos(Directory dir, String segmentFileName, SegmentInfos infos, IOContext context)
|
||||
public IndexOutput writeInfos(Directory dir, String segmentFileName, String codecID, SegmentInfos infos, IOContext context)
|
||||
throws IOException {
|
||||
IndexOutput out = createOutput(dir, segmentFileName, new IOContext(new FlushInfo(infos.size(), infos.totalDocCount())));
|
||||
boolean success = false;
|
||||
try {
|
||||
out.writeInt(FORMAT_CURRENT); // write FORMAT
|
||||
out.writeString(codecID); // write codecID
|
||||
out.writeLong(infos.version);
|
||||
out.writeInt(infos.counter); // write counter
|
||||
out.writeLong(infos.getGlobalFieldMapVersion());
|
||||
|
|
|
@ -0,0 +1,32 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
public abstract class DocValuesFormat {
|
||||
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
|
||||
public abstract PerDocValues docsProducer(SegmentReadState state) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
|
@ -65,19 +65,19 @@ public abstract class DocValuesReaderBase extends PerDocValues {
|
|||
|
||||
// Only opens files... doesn't actually load any values
|
||||
protected TreeMap<String, IndexDocValues> load(FieldInfos fieldInfos,
|
||||
String segment, int docCount, Directory dir, int codecId, IOContext context)
|
||||
String segment, int docCount, Directory dir, IOContext context)
|
||||
throws IOException {
|
||||
TreeMap<String, IndexDocValues> values = new TreeMap<String, IndexDocValues>();
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (codecId == fieldInfo.getCodecId() && fieldInfo.hasDocValues()) {
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
final String field = fieldInfo.name;
|
||||
// TODO can we have a compound file per segment and codec for
|
||||
// docvalues?
|
||||
final String id = DefaultDocValuesConsumer.docValuesId(segment,
|
||||
codecId, fieldInfo.number);
|
||||
fieldInfo.number);
|
||||
values.put(field,
|
||||
loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), context));
|
||||
}
|
||||
|
|
|
@ -33,19 +33,19 @@ import org.apache.lucene.util.Counter;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class DocValuesWriterBase extends PerDocConsumer {
|
||||
private final String segmentName;
|
||||
private final int codecId;
|
||||
protected final String segmentName;
|
||||
protected final String segmentSuffix;
|
||||
private final Counter bytesUsed;
|
||||
private final IOContext context;
|
||||
protected final IOContext context;
|
||||
|
||||
protected DocValuesWriterBase(PerDocWriteState state) {
|
||||
this.segmentName = state.segmentName;
|
||||
this.codecId = state.codecId;
|
||||
this.segmentSuffix = state.segmentSuffix;
|
||||
this.bytesUsed = state.bytesUsed;
|
||||
this.context = state.context;
|
||||
}
|
||||
|
||||
protected abstract Directory getDirectory();
|
||||
protected abstract Directory getDirectory() throws IOException;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
@ -54,12 +54,12 @@ public abstract class DocValuesWriterBase extends PerDocConsumer {
|
|||
@Override
|
||||
public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
|
||||
return Writer.create(field.getDocValues(),
|
||||
docValuesId(segmentName, codecId, field.number),
|
||||
docValuesId(segmentName, field.number),
|
||||
getDirectory(), getComparator(), bytesUsed, context);
|
||||
}
|
||||
|
||||
public static String docValuesId(String segmentsName, int codecID, int fieldId) {
|
||||
return segmentsName + "_" + codecID + "-" + fieldId;
|
||||
public static String docValuesId(String segmentsName, int fieldId) {
|
||||
return segmentsName + "_" + fieldId;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,35 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Controls the format of stored fields/termvectors/...
|
||||
*/
|
||||
public abstract class FieldsFormat {
|
||||
public abstract FieldsReader fieldsReader(Directory directory, String segment, FieldInfos fn, IOContext context, int docStoreOffset, int size) throws IOException;
|
||||
public abstract FieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
|
@ -34,23 +34,4 @@ import org.apache.lucene.index.Terms;
|
|||
|
||||
public abstract class FieldsProducer extends Fields implements Closeable {
|
||||
public abstract void close() throws IOException;
|
||||
|
||||
public static final FieldsProducer EMPTY = new FieldsProducer() {
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsEnum iterator() throws IOException {
|
||||
return FieldsEnum.EMPTY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
|
|
@ -68,14 +68,14 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
// start of the field info data
|
||||
protected long dirOffset;
|
||||
|
||||
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId, IOContext context)
|
||||
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
|
||||
throws IOException {
|
||||
|
||||
this.termComp = termComp;
|
||||
|
||||
assert indexDivisor == -1 || indexDivisor > 0;
|
||||
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
|
||||
|
||||
boolean success = false;
|
||||
|
||||
|
@ -387,17 +387,8 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, id, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
public static void getIndexExtensions(Collection<String> extensions) {
|
||||
extensions.add(FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Collection<String> extensions) {
|
||||
getIndexExtensions(extensions);
|
||||
public static void files(Directory dir, SegmentInfo info, String segmentSuffix, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -56,7 +56,7 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
private final FieldInfos fieldInfos; // unread
|
||||
|
||||
public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_INDEX_EXTENSION);
|
||||
termIndexInterval = state.termIndexInterval;
|
||||
out = state.directory.createOutput(indexFileName, state.context);
|
||||
boolean success = false;
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.lucene.index.values.ValueType;
|
|||
* this convert field values into a Codec specific format during indexing.
|
||||
* <p>
|
||||
* The {@link PerDocConsumer} API is accessible through the
|
||||
* {@link Codec} - API providing per field consumers and producers for inverted
|
||||
* {@link PostingsFormat} - API providing per field consumers and producers for inverted
|
||||
* data (terms, postings) as well as per-document data.
|
||||
*
|
||||
* @lucene.experimental
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.index.values.IndexDocValues;
|
|||
* {@link PerDocConsumer} counterpart.
|
||||
* <p>
|
||||
* The {@link PerDocValues} API is accessible through the
|
||||
* {@link Codec} - API providing per field consumers and producers for inverted
|
||||
* {@link PostingsFormat} - API providing per field consumers and producers for inverted
|
||||
* data (terms, postings) as well as per-document data.
|
||||
*
|
||||
* @lucene.experimental
|
||||
|
|
|
@ -0,0 +1,55 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Provides a {@link PostingsReaderBase} and {@link
|
||||
* PostingsWriterBase}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
// TODO: find a better name; this defines the API that the
|
||||
// terms dict impls use to talk to a postings impl.
|
||||
// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer
|
||||
|
||||
// can we clean this up and do this some other way?
|
||||
// refactor some of these classes and use covariant return?
|
||||
public abstract class PostingsBaseFormat {
|
||||
|
||||
/** Unique name that's used to retrieve this codec when
|
||||
* reading the index */
|
||||
public final String name;
|
||||
|
||||
protected PostingsBaseFormat(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
public abstract PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException;
|
||||
|
||||
public abstract PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException;
|
||||
|
||||
public abstract void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException;
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.util.NamedSPILoader;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public abstract class PostingsFormat implements NamedSPILoader.NamedSPI {
|
||||
|
||||
private static final NamedSPILoader<PostingsFormat> loader =
|
||||
new NamedSPILoader<PostingsFormat>(PostingsFormat.class);
|
||||
|
||||
public static final PostingsFormat[] EMPTY = new PostingsFormat[0];
|
||||
/** Unique name that's used to retrieve this format when
|
||||
* reading the index.
|
||||
*/
|
||||
private final String name;
|
||||
|
||||
protected PostingsFormat(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
/** Writes a new segment */
|
||||
public abstract FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException;
|
||||
|
||||
/** Reads a segment. NOTE: by the time this call
|
||||
* returns, it must hold open any files it will need to
|
||||
* use; else, those files may be deleted. */
|
||||
public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
|
||||
|
||||
/**
|
||||
* Gathers files associated with this segment
|
||||
*
|
||||
* @param dir the {@link Directory} this segment was written to
|
||||
* @param segmentInfo the {@link SegmentInfo} for this segment
|
||||
* @param segmentSuffix the format's suffix within this segment
|
||||
* @param files the of files to add the codec files to.
|
||||
*/
|
||||
public abstract void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "PostingsFormat(name=" + name + ")";
|
||||
}
|
||||
|
||||
/** looks up a format by name */
|
||||
public static PostingsFormat forName(String name) {
|
||||
return loader.lookup(name);
|
||||
}
|
||||
|
||||
/** returns a list of all available format names */
|
||||
public static Set<String> availablePostingsFormats() {
|
||||
return loader.availableServices();
|
||||
}
|
||||
|
||||
}
|
|
@ -36,6 +36,9 @@ import org.apache.lucene.util.Bits;
|
|||
* time.
|
||||
* @lucene.experimental */
|
||||
|
||||
// TODO: find a better name; this defines the API that the
|
||||
// terms dict impls use to talk to a postings impl.
|
||||
// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer
|
||||
public abstract class PostingsReaderBase implements Closeable {
|
||||
|
||||
public abstract void init(IndexInput termsIn) throws IOException;
|
||||
|
|
|
@ -27,6 +27,9 @@ import org.apache.lucene.index.FieldInfo;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
// TODO: find a better name; this defines the API that the
|
||||
// terms dict impls use to talk to a postings impl.
|
||||
// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer
|
||||
public abstract class PostingsWriterBase extends PostingsConsumer implements Closeable {
|
||||
|
||||
public abstract void start(IndexOutput termsOut) throws IOException;
|
||||
|
|
|
@ -0,0 +1,37 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Expert: Controls the format of the segments file.
|
||||
* Note, this isn't a per-segment file, if you change the format, other versions
|
||||
* of lucene won't be able to read it, yackedy schmackedy
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
// TODO: would be great to handle this situation better.
|
||||
// ideally a custom implementation could implement two-phase commit differently,
|
||||
// (e.g. atomic rename), and ideally all versions of lucene could still read it.
|
||||
// but this is just reflecting reality as it is today...
|
||||
//
|
||||
// also, perhaps the name should change (to cover all global files like .fnx?)
|
||||
// then again, maybe we can just remove that file...
|
||||
public abstract class SegmentInfosFormat {
|
||||
public abstract SegmentInfosReader getSegmentInfosReader();
|
||||
public abstract SegmentInfosWriter getSegmentInfosWriter();
|
||||
}
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.SegmentInfos;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
|
@ -33,9 +34,9 @@ public abstract class SegmentInfosReader {
|
|||
* Read {@link SegmentInfos} data from a directory.
|
||||
* @param directory directory to read from
|
||||
* @param segmentsFileName name of the "segments_N" file
|
||||
* @param codecs current codecs
|
||||
* @param header input of "segments_N" file after reading preamble
|
||||
* @param infos empty instance to be populated with data
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract void read(Directory directory, String segmentsFileName, CodecProvider codecs, SegmentInfos infos, IOContext context) throws IOException;
|
||||
public abstract void read(Directory directory, String segmentsFileName, ChecksumIndexInput header, SegmentInfos infos, IOContext context) throws IOException;
|
||||
}
|
||||
|
|
|
@ -42,13 +42,13 @@ public abstract class SegmentInfosWriter {
|
|||
* phase commit" operations as described above.
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, SegmentInfos infos, IOContext context) throws IOException;
|
||||
public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, String codecID, SegmentInfos infos, IOContext context) throws IOException;
|
||||
|
||||
/**
|
||||
* First phase of the two-phase commit - ensure that all output can be
|
||||
* successfully written out.
|
||||
* @param out an instance of {@link IndexOutput} returned from a previous
|
||||
* call to {@link #writeInfos(Directory, String, SegmentInfos, IOContext)}.
|
||||
* call to {@link #writeInfos(Directory, String, String, SegmentInfos, IOContext)}.
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract void prepareCommit(IndexOutput out) throws IOException;
|
||||
|
@ -57,7 +57,7 @@ public abstract class SegmentInfosWriter {
|
|||
* Second phase of the two-phase commit. In this step the output should be
|
||||
* finalized and closed.
|
||||
* @param out an instance of {@link IndexOutput} returned from a previous
|
||||
* call to {@link #writeInfos(Directory, String, SegmentInfos, IOContext)}.
|
||||
* call to {@link #writeInfos(Directory, String, String, SegmentInfos, IOContext)}.
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract void finishCommit(IndexOutput out) throws IOException;
|
||||
|
|
|
@ -45,8 +45,6 @@ public abstract class TermsIndexReaderBase implements Closeable {
|
|||
|
||||
public abstract void close() throws IOException;
|
||||
|
||||
public abstract void getExtensions(Collection<String> extensions);
|
||||
|
||||
public abstract boolean supportsOrd();
|
||||
|
||||
public abstract int getDivisor();
|
||||
|
|
|
@ -57,9 +57,9 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
protected long dirOffset;
|
||||
|
||||
final String segment;
|
||||
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId, IOContext context)
|
||||
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String segmentSuffix, IOContext context)
|
||||
throws IOException {
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(context, true));
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(context, true));
|
||||
this.segment = segment;
|
||||
boolean success = false;
|
||||
assert indexDivisor == -1 || indexDivisor > 0;
|
||||
|
@ -215,17 +215,8 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, id, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
public static void getIndexExtensions(Collection<String> extensions) {
|
||||
extensions.add(VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Collection<String> extensions) {
|
||||
getIndexExtensions(extensions);
|
||||
public static void files(Directory dir, SegmentInfo info, String segmentSuffix, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -158,7 +158,7 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
// in the extremes.
|
||||
|
||||
public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException {
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_INDEX_EXTENSION);
|
||||
out = state.directory.createOutput(indexFileName, state.context);
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -0,0 +1,90 @@
|
|||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Supports the Lucene 3.x index format (readonly)
|
||||
*/
|
||||
public class Lucene3xCodec extends Codec {
|
||||
public Lucene3xCodec() {
|
||||
super("Lucene3x");
|
||||
}
|
||||
|
||||
private final PostingsFormat postingsFormat = new Lucene3xPostingsFormat();
|
||||
|
||||
// TODO: this should really be a different impl
|
||||
private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
|
||||
|
||||
// TODO: this should really be a different impl
|
||||
// also if we want preflex to *really* be read-only it should throw exception for the writer?
|
||||
// this way IR.commit fails on delete/undelete/setNorm/etc ?
|
||||
private final SegmentInfosFormat infosFormat = new DefaultSegmentInfosFormat();
|
||||
|
||||
// 3.x doesn't support docvalues
|
||||
private final DocValuesFormat docValuesFormat = new DocValuesFormat() {
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {}
|
||||
};
|
||||
|
||||
@Override
|
||||
public PostingsFormat postingsFormat() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsFormat fieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentInfosFormat segmentInfosFormat() {
|
||||
return infosFormat;
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -51,7 +51,7 @@ import org.apache.lucene.util.UnicodeUtil;
|
|||
* @deprecated (4.0)
|
||||
*/
|
||||
@Deprecated
|
||||
public class PreFlexFields extends FieldsProducer {
|
||||
public class Lucene3xFields extends FieldsProducer {
|
||||
|
||||
private static final boolean DEBUG_SURROGATES = false;
|
||||
|
||||
|
@ -68,7 +68,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
private final IOContext context;
|
||||
private Directory cfsReader;
|
||||
|
||||
public PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
|
||||
public Lucene3xFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
|
||||
throws IOException {
|
||||
|
||||
si = info;
|
||||
|
@ -95,7 +95,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
|
||||
// make sure that all index files have been read or are kept open
|
||||
// so that if an index update removes them we'll still have them
|
||||
freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.FREQ_EXTENSION), context);
|
||||
freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION), context);
|
||||
boolean anyProx = false;
|
||||
for (FieldInfo fi : fieldInfos) {
|
||||
if (fi.isIndexed) {
|
||||
|
@ -108,7 +108,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
}
|
||||
|
||||
if (anyProx) {
|
||||
proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.PROX_EXTENSION), context);
|
||||
proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION), context);
|
||||
} else {
|
||||
proxStream = null;
|
||||
}
|
||||
|
@ -136,16 +136,16 @@ public class PreFlexFields extends FieldsProducer {
|
|||
}
|
||||
|
||||
static void files(Directory dir, SegmentInfo info, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.TERMS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.TERMS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.FREQ_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.TERMS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION));
|
||||
if (info.getHasProx()) {
|
||||
// LUCENE-1739: for certain versions of 2.9-dev,
|
||||
// hasProx would be incorrectly computed during
|
||||
// indexing as true, and then stored into the segments
|
||||
// file, when it should have been false. So we do the
|
||||
// extra check, here:
|
||||
final String prx = IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.PROX_EXTENSION);
|
||||
final String prx = IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION);
|
||||
if (dir.fileExists(prx)) {
|
||||
files.add(prx);
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -21,15 +21,12 @@ import java.util.Set;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
|
||||
/** Codec that reads the pre-flex-indexing postings
|
||||
* format. It does not provide a writer because newly
|
||||
|
@ -40,7 +37,7 @@ import org.apache.lucene.index.codecs.PerDocValues;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
@Deprecated
|
||||
public class PreFlexCodec extends Codec {
|
||||
public class Lucene3xPostingsFormat extends PostingsFormat {
|
||||
|
||||
/** Extension of terms file */
|
||||
public static final String TERMS_EXTENSION = "tis";
|
||||
|
@ -54,8 +51,8 @@ public class PreFlexCodec extends Codec {
|
|||
/** Extension of prox postings file */
|
||||
public static final String PROX_EXTENSION = "prx";
|
||||
|
||||
public PreFlexCodec() {
|
||||
super("PreFlex");
|
||||
public Lucene3xPostingsFormat() {
|
||||
super("Lucene3x");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -65,30 +62,12 @@ public class PreFlexCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
|
||||
return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, int id, Set<String> files) throws IOException {
|
||||
// preflex fields have no codec ID - we ignore it here
|
||||
PreFlexFields.files(dir, info, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
extensions.add(FREQ_EXTENSION);
|
||||
extensions.add(PROX_EXTENSION);
|
||||
extensions.add(TERMS_EXTENSION);
|
||||
extensions.add(TERMS_INDEX_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return null;
|
||||
public void files(Directory dir, SegmentInfo info, String segmentSuffix, Set<String> files) throws IOException {
|
||||
// preflex fields have no segmentSuffix - we ignore it here
|
||||
Lucene3xFields.files(dir, info, files);
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -23,7 +23,7 @@ import org.apache.lucene.index.FieldInfo;
|
|||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.codecs.standard.DefaultSkipListReader;
|
||||
import org.apache.lucene.index.codecs.lucene40.DefaultSkipListReader;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -109,7 +109,7 @@ public final class TermInfosReader {
|
|||
segment = seg;
|
||||
fieldInfos = fis;
|
||||
|
||||
origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_EXTENSION),
|
||||
origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_EXTENSION),
|
||||
context), fieldInfos, false);
|
||||
size = origEnum.size;
|
||||
|
||||
|
@ -118,7 +118,7 @@ public final class TermInfosReader {
|
|||
// Load terms index
|
||||
totalIndexInterval = origEnum.indexInterval * indexDivisor;
|
||||
|
||||
final String indexFileName = IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_INDEX_EXTENSION);
|
||||
final String indexFileName = IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION);
|
||||
final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(indexFileName,
|
||||
context), fieldInfos, true);
|
||||
|
||||
|
@ -228,8 +228,8 @@ public final class TermInfosReader {
|
|||
|
||||
// optimize sequential access: first try scanning cached enum w/o seeking
|
||||
if (enumerator.term() != null // term is at or past current
|
||||
&& ((enumerator.prev() != null && compareAsUTF16(term, enumerator.prev())> 0)
|
||||
|| compareAsUTF16(term, enumerator.term()) >= 0)) {
|
||||
&& ((enumerator.prev() != null && compareAsUTF16(term, enumerator.prev())> 0)
|
||||
|| compareAsUTF16(term, enumerator.term()) >= 0)) {
|
||||
int enumOffset = (int)(enumerator.position/totalIndexInterval)+1;
|
||||
if (indexLength == enumOffset // but before end of block
|
||||
|| index.compareTo(term, enumOffset) < 0) {
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.preflex;
|
||||
package org.apache.lucene.index.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs.lucene40;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs.lucene40;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -0,0 +1,85 @@
|
|||
package org.apache.lucene.index.codecs.lucene40;
|
||||
|
||||
/**
|
||||
* 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.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 4.0 index format, with configurable per-field postings formats
|
||||
* and using {@link DefaultFieldsFormat} for stored fields and {@link
|
||||
* DefaultDocValuesFormat} for doc values.
|
||||
*
|
||||
* @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 Lucene40Codec extends Codec {
|
||||
private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
|
||||
private final DocValuesFormat docValuesFormat = new DefaultDocValuesFormat();
|
||||
private final SegmentInfosFormat infosFormat = new DefaultSegmentInfosFormat();
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return Lucene40Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
public Lucene40Codec() {
|
||||
super("Lucene40");
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsFormat fieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsFormat postingsFormat() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentInfosFormat segmentInfosFormat() {
|
||||
return infosFormat;
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene40"
|
||||
*/
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return defaultFormat;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene40");
|
||||
}
|
|
@ -0,0 +1,58 @@
|
|||
package org.apache.lucene.index.codecs.lucene40;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.codecs.PostingsBaseFormat;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Provides a {@link PostingsReaderBase} and {@link
|
||||
* PostingsWriterBase}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
// TODO: should these also be named / looked up via SPI?
|
||||
public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
|
||||
|
||||
public Lucene40PostingsBaseFormat() {
|
||||
super("Lucene40");
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
|
||||
return new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
|
||||
return new Lucene40PostingsWriter(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs.lucene40;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -20,7 +20,6 @@ package org.apache.lucene.index.codecs.standard;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -28,28 +27,28 @@ import org.apache.lucene.index.codecs.PostingsReaderBase;
|
|||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.codecs.BlockTreeTermsReader;
|
||||
import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** Default codec.
|
||||
* @lucene.experimental */
|
||||
public class StandardCodec extends Codec {
|
||||
|
||||
// TODO: this class could be created by wrapping
|
||||
// BlockTreeTermsDict around Lucene40PostingsBaseFormat; ie
|
||||
// we should not duplicate the code from that class here:
|
||||
public class Lucene40PostingsFormat extends PostingsFormat {
|
||||
|
||||
private final int minBlockSize;
|
||||
private final int maxBlockSize;
|
||||
|
||||
public StandardCodec() {
|
||||
public Lucene40PostingsFormat() {
|
||||
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
|
||||
}
|
||||
|
||||
public StandardCodec(int minBlockSize, int maxBlockSize) {
|
||||
super("Standard");
|
||||
public Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
|
||||
super("Lucene40");
|
||||
this.minBlockSize = minBlockSize;
|
||||
assert minBlockSize > 1;
|
||||
this.maxBlockSize = maxBlockSize;
|
||||
|
@ -57,7 +56,7 @@ public class StandardCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
PostingsWriterBase docs = new StandardPostingsWriter(state);
|
||||
PostingsWriterBase docs = new Lucene40PostingsWriter(state);
|
||||
|
||||
// TODO: should we make the terms index more easily
|
||||
// pluggable? Ie so that this codec would record which
|
||||
|
@ -79,7 +78,7 @@ public class StandardCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
|
||||
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
@ -89,7 +88,7 @@ public class StandardCodec extends Codec {
|
|||
state.segmentInfo.name,
|
||||
postings,
|
||||
state.context,
|
||||
state.codecId,
|
||||
state.segmentSuffix,
|
||||
state.termsIndexDivisor);
|
||||
success = true;
|
||||
return ret;
|
||||
|
@ -107,36 +106,13 @@ public class StandardCodec extends Codec {
|
|||
static final String PROX_EXTENSION = "prx";
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecID, files);
|
||||
BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
getStandardExtensions(extensions);
|
||||
}
|
||||
|
||||
public static void getStandardExtensions(Set<String> extensions) {
|
||||
extensions.add(FREQ_EXTENSION);
|
||||
extensions.add(PROX_EXTENSION);
|
||||
BlockTreeTermsReader.getExtensions(extensions);
|
||||
DefaultDocValuesConsumer.getExtensions(extensions);
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
BlockTreeTermsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return name + "(minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new DefaultDocValuesConsumer(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new DefaultDocValuesProducer(state);
|
||||
return getName() + "(minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs.lucene40;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -42,7 +42,7 @@ import org.apache.lucene.util.CodecUtil;
|
|||
* postings format.
|
||||
* @lucene.experimental */
|
||||
|
||||
public class StandardPostingsReader extends PostingsReaderBase {
|
||||
public class Lucene40PostingsReader extends PostingsReaderBase {
|
||||
|
||||
private final IndexInput freqIn;
|
||||
private final IndexInput proxIn;
|
||||
|
@ -54,14 +54,14 @@ public class StandardPostingsReader extends PostingsReaderBase {
|
|||
|
||||
// private String segment;
|
||||
|
||||
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
|
||||
public Lucene40PostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
|
||||
ioContext);
|
||||
// this.segment = segmentInfo.name;
|
||||
if (segmentInfo.getHasProx()) {
|
||||
boolean success = false;
|
||||
try {
|
||||
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
|
||||
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION),
|
||||
ioContext);
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -74,10 +74,10 @@ public class StandardPostingsReader extends PostingsReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.FREQ_EXTENSION));
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION));
|
||||
if (segmentInfo.getHasProx()) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.PROX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -85,8 +85,8 @@ public class StandardPostingsReader extends PostingsReaderBase {
|
|||
public void init(IndexInput termsIn) throws IOException {
|
||||
|
||||
// Make sure we are talking to the matching past writer
|
||||
CodecUtil.checkHeader(termsIn, StandardPostingsWriter.CODEC,
|
||||
StandardPostingsWriter.VERSION_START, StandardPostingsWriter.VERSION_START);
|
||||
CodecUtil.checkHeader(termsIn, Lucene40PostingsWriter.CODEC,
|
||||
Lucene40PostingsWriter.VERSION_START, Lucene40PostingsWriter.VERSION_START);
|
||||
|
||||
skipInterval = termsIn.readInt();
|
||||
maxSkipLevels = termsIn.readInt();
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs.lucene40;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -38,8 +38,8 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.CodecUtil;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public final class StandardPostingsWriter extends PostingsWriterBase {
|
||||
final static String CODEC = "StandardPostingsWriter";
|
||||
public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
||||
final static String CODEC = "Lucene40PostingsWriter";
|
||||
|
||||
//private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
|
||||
|
||||
|
@ -81,21 +81,21 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
|
|||
|
||||
// private String segment;
|
||||
|
||||
public StandardPostingsWriter(SegmentWriteState state) throws IOException {
|
||||
public Lucene40PostingsWriter(SegmentWriteState state) throws IOException {
|
||||
this(state, DEFAULT_SKIP_INTERVAL);
|
||||
}
|
||||
|
||||
public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
|
||||
public Lucene40PostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
|
||||
super();
|
||||
this.skipInterval = skipInterval;
|
||||
this.skipMinimum = skipInterval; /* set to the same for now */
|
||||
// this.segment = state.segmentName;
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION);
|
||||
freqOut = state.directory.createOutput(fileName, state.context);
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
// At least one field does not omit TF, so create the
|
||||
// prox file
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION);
|
||||
proxOut = state.directory.createOutput(fileName, state.context);
|
||||
} else {
|
||||
// Every field omits TF so we will write no prox file
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue