mirror of https://github.com/apache/lucene.git
LUCENE-3728: better handling of files inside/outside CFS by codec
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1238015 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
8e0a95de17
|
@ -186,7 +186,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) {
|
||||
public static void files(SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockTermsWriter.TERMS_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -200,7 +200,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) {
|
||||
public static void files(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));
|
||||
}
|
||||
|
|
|
@ -20,10 +20,9 @@ package org.apache.lucene.codecs;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.util.NamedSPILoader;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Encodes/decodes an inverted index segment
|
||||
|
@ -47,24 +46,22 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
/** Populates <code>files</code> with all filenames needed for
|
||||
* the <code>info</code> segment.
|
||||
*/
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
assert (dir instanceof CompoundFileDirectory) == false;
|
||||
postingsFormat().files(dir, info, "", files);
|
||||
storedFieldsFormat().files(dir, info, files);
|
||||
termVectorsFormat().files(dir, info, files);
|
||||
fieldInfosFormat().files(dir, info, files);
|
||||
// TODO: segmentInfosFormat should be allowed to declare additional files
|
||||
// if it wants, in addition to segments_N
|
||||
docValuesFormat().files(dir, info, files);
|
||||
normsFormat().files(dir, info, files);
|
||||
}
|
||||
|
||||
/** Populates <code>files</code> with any filenames that are
|
||||
* stored outside of CFS for the <code>info</code> segment.
|
||||
*/
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
liveDocsFormat().separateFiles(dir, info, files);
|
||||
normsFormat().separateFiles(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getUseCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
} else {
|
||||
postingsFormat().files(info, "", files);
|
||||
storedFieldsFormat().files(info, files);
|
||||
termVectorsFormat().files(info, files);
|
||||
fieldInfosFormat().files(info, files);
|
||||
// TODO: segmentInfosFormat should be allowed to declare additional files
|
||||
// if it wants, in addition to segments_N
|
||||
docValuesFormat().files(info, files);
|
||||
normsFormat().files(info, files);
|
||||
}
|
||||
// never inside CFS
|
||||
liveDocsFormat().files(info, files);
|
||||
}
|
||||
|
||||
/** Encodes/decodes postings */
|
||||
|
|
|
@ -23,10 +23,9 @@ 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 PerDocProducer docsProducer(SegmentReadState state) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
|
@ -29,5 +28,5 @@ import org.apache.lucene.store.Directory;
|
|||
public abstract class FieldInfosFormat {
|
||||
public abstract FieldInfosReader getFieldInfosReader() throws IOException;
|
||||
public abstract FieldInfosWriter getFieldInfosWriter() throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
||||
|
|
|
@ -387,7 +387,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, String segmentSuffix, Collection<String> files) {
|
||||
public static void files(SegmentInfo info, String segmentSuffix, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -37,5 +37,5 @@ public abstract class LiveDocsFormat {
|
|||
public abstract Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) throws IOException;
|
||||
/** writes bits to a file */
|
||||
public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException;
|
||||
public abstract void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ 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;
|
||||
|
||||
/**
|
||||
* format for normalization factors
|
||||
|
@ -31,21 +30,5 @@ import org.apache.lucene.store.Directory;
|
|||
public abstract class NormsFormat {
|
||||
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
|
||||
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
|
||||
/**
|
||||
* Note: this should not be overridden!
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {};
|
||||
|
||||
/**
|
||||
* Note: this should not be overridden!
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
public PerDocProducer docsProducer(SegmentReadState state, Directory separateNormsDir) throws IOException {
|
||||
return docsProducer(state);
|
||||
}
|
||||
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ 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
|
||||
|
@ -51,5 +50,5 @@ public abstract class PostingsBaseFormat {
|
|||
|
||||
public abstract PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException;
|
||||
|
||||
public abstract void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException;
|
||||
public abstract void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException;
|
||||
}
|
||||
|
|
|
@ -25,8 +25,6 @@ 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 {
|
||||
|
||||
|
@ -59,12 +57,11 @@ public abstract class PostingsFormat implements NamedSPILoader.NamedSPI {
|
|||
/**
|
||||
* 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;
|
||||
public abstract void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
|
|
|
@ -31,5 +31,5 @@ import org.apache.lucene.store.IOContext;
|
|||
public abstract class StoredFieldsFormat {
|
||||
public abstract StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException;
|
||||
public abstract StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
||||
|
|
|
@ -31,5 +31,5 @@ import org.apache.lucene.store.IOContext;
|
|||
public abstract class TermVectorsFormat {
|
||||
public abstract TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException;
|
||||
public abstract TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
||||
|
|
|
@ -217,7 +217,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, String segmentSuffix, Collection<String> files) {
|
||||
public static void files(SegmentInfo info, String segmentSuffix, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -83,9 +83,8 @@ class AppendingPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
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);
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,16 +24,14 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
@ -43,7 +41,9 @@ import org.apache.lucene.util.MutableBits;
|
|||
|
||||
/**
|
||||
* Supports the Lucene 3.x index format (readonly)
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene3xCodec extends Codec {
|
||||
public Lucene3xCodec() {
|
||||
super("Lucene3x");
|
||||
|
@ -51,13 +51,7 @@ public class Lucene3xCodec extends Codec {
|
|||
|
||||
private final PostingsFormat postingsFormat = new Lucene3xPostingsFormat();
|
||||
|
||||
// TODO: this should really be a different impl
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat() {
|
||||
@Override
|
||||
public StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
};
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene3xStoredFieldsFormat();
|
||||
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene3xTermVectorsFormat();
|
||||
|
||||
|
@ -65,7 +59,10 @@ public class Lucene3xCodec extends Codec {
|
|||
|
||||
private final SegmentInfosFormat infosFormat = new Lucene3xSegmentInfosFormat();
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene3xNormsFormat();
|
||||
private final Lucene3xNormsFormat normsFormat = new Lucene3xNormsFormat();
|
||||
|
||||
/** Extension of compound file for doc store files*/
|
||||
static final String COMPOUND_FILE_STORE_EXTENSION = "cfx";
|
||||
|
||||
// TODO: this should really be a different impl
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat() {
|
||||
|
@ -88,7 +85,7 @@ public class Lucene3xCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {}
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {}
|
||||
};
|
||||
|
||||
@Override
|
||||
|
@ -122,7 +119,7 @@ public class Lucene3xCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public NormsFormat normsFormat() {
|
||||
public Lucene3xNormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
|
||||
|
@ -130,4 +127,31 @@ public class Lucene3xCodec extends Codec {
|
|||
public LiveDocsFormat liveDocsFormat() {
|
||||
return liveDocsFormat;
|
||||
}
|
||||
|
||||
// overrides the default implementation in codec.java to handle CFS without CFE,
|
||||
// shared doc stores, compound doc stores, separate norms, etc
|
||||
@Override
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getUseCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
} else {
|
||||
postingsFormat().files(info, "", files);
|
||||
storedFieldsFormat().files(info, files);
|
||||
termVectorsFormat().files(info, files);
|
||||
fieldInfosFormat().files(info, files);
|
||||
// TODO: segmentInfosFormat should be allowed to declare additional files
|
||||
// if it wants, in addition to segments_N
|
||||
docValuesFormat().files(info, files);
|
||||
normsFormat().files(info, files);
|
||||
}
|
||||
// never inside CFS
|
||||
liveDocsFormat().files(info, files);
|
||||
normsFormat().separateFiles(info, files);
|
||||
|
||||
// shared docstores: these guys check the hair
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
storedFieldsFormat().files(info, files);
|
||||
termVectorsFormat().files(info, files);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.lucene.codecs.FieldInfosFormat;
|
|||
import org.apache.lucene.codecs.FieldInfosReader;
|
||||
import org.apache.lucene.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Lucene3x ReadOnly FieldInfosFromat implementation
|
||||
|
@ -47,7 +46,7 @@ public class Lucene3xFieldInfosFormat extends FieldInfosFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xFieldInfosReader.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xFieldInfosReader.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,7 +34,9 @@ import org.apache.lucene.store.IndexInput;
|
|||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene3xFieldInfosReader extends FieldInfosReader {
|
||||
/** Extension of field infos */
|
||||
static final String FIELD_INFOS_EXTENSION = "fnm";
|
||||
|
@ -117,7 +119,7 @@ public class Lucene3xFieldInfosReader extends FieldInfosReader {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", FIELD_INFOS_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -133,7 +133,7 @@ public class Lucene3xFields extends FieldsProducer {
|
|||
return true;
|
||||
}
|
||||
|
||||
static void files(Directory dir, SegmentInfo info, Collection<String> files) throws IOException {
|
||||
static void files(SegmentInfo info, Collection<String> files) throws IOException {
|
||||
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));
|
||||
|
@ -144,7 +144,7 @@ public class Lucene3xFields extends FieldsProducer {
|
|||
// file, when it should have been false. So we do the
|
||||
// extra check, here:
|
||||
final String prx = IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION);
|
||||
if (dir.fileExists(prx)) {
|
||||
if (info.dir.fileExists(prx)) {
|
||||
files.add(prx);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.codecs.PerDocProducer;
|
|||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Lucene3x ReadOnly NormsFormat implementation
|
||||
|
@ -39,13 +38,12 @@ public class Lucene3xNormsFormat extends NormsFormat {
|
|||
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xNormsProducer.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xNormsProducer.files(info, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xNormsProducer.separateFiles(dir, info, files);
|
||||
public void separateFiles(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xNormsProducer.separateFiles(info, files);
|
||||
}
|
||||
|
||||
|
||||
|
@ -56,12 +54,6 @@ public class Lucene3xNormsFormat extends NormsFormat {
|
|||
|
||||
@Override
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return docsProducer(state, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocProducer docsProducer(SegmentReadState state,
|
||||
Directory separateNormsDir) throws IOException {
|
||||
return new Lucene3xNormsProducer(state.dir, state.segmentInfo, state.fieldInfos, state.context, separateNormsDir);
|
||||
return new Lucene3xNormsProducer(state.dir, state.segmentInfo, state.fieldInfos, state.context);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,7 +43,9 @@ import org.apache.lucene.util.StringHelper;
|
|||
/**
|
||||
* Reads Lucene 3.x norms format and exposes it via DocValues API
|
||||
* @lucene.experimental
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
class Lucene3xNormsProducer extends PerDocProducer {
|
||||
|
||||
/** norms header placeholder */
|
||||
|
@ -52,9 +54,7 @@ class Lucene3xNormsProducer extends PerDocProducer {
|
|||
/** Extension of norms file */
|
||||
static final String NORMS_EXTENSION = "nrm";
|
||||
|
||||
/** Extension of separate norms file
|
||||
* @deprecated */
|
||||
@Deprecated
|
||||
/** Extension of separate norms file */
|
||||
static final String SEPARATE_NORMS_EXTENSION = "s";
|
||||
|
||||
final Map<String,NormsDocValues> norms = new HashMap<String,NormsDocValues>();
|
||||
|
@ -67,7 +67,8 @@ class Lucene3xNormsProducer extends PerDocProducer {
|
|||
|
||||
// note: just like segmentreader in 3.x, we open up all the files here (including separate norms) up front.
|
||||
// but we just don't do any seeks or reading yet.
|
||||
public Lucene3xNormsProducer(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context, Directory separateNormsDir) throws IOException {
|
||||
public Lucene3xNormsProducer(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context) throws IOException {
|
||||
Directory separateNormsDir = info.dir; // separate norms are never inside CFS
|
||||
maxdoc = info.docCount;
|
||||
String segmentName = info.name;
|
||||
Map<Integer,Long> normGen = info.getNormGen();
|
||||
|
@ -192,19 +193,17 @@ class Lucene3xNormsProducer extends PerDocProducer {
|
|||
|
||||
}
|
||||
|
||||
static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
// TODO: This is what SI always did... but we can do this cleaner?
|
||||
// like first FI that has norms but doesn't have separate norms?
|
||||
final String normsFileName = IndexFileNames.segmentFileName(info.name, "", NORMS_EXTENSION);
|
||||
if (dir.fileExists(normsFileName)) {
|
||||
if (info.dir.fileExists(normsFileName)) {
|
||||
// only needed to do this in 3x - 4x can decide if the norms are present
|
||||
files.add(normsFileName);
|
||||
}
|
||||
}
|
||||
|
||||
/** @deprecated */
|
||||
@Deprecated
|
||||
static void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
static void separateFiles(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Map<Integer,Long> normGen = info.getNormGen();
|
||||
if (normGen != null) {
|
||||
for (Entry<Integer,Long> entry : normGen.entrySet()) {
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.codecs.lucene3x;
|
|||
import java.util.Set;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
|
@ -66,8 +65,8 @@ public class Lucene3xPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, String segmentSuffix, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo info, String segmentSuffix, Set<String> files) throws IOException {
|
||||
// preflex fields have no segmentSuffix - we ignore it here
|
||||
Lucene3xFields.files(dir, info, files);
|
||||
Lucene3xFields.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,7 +37,9 @@ import org.apache.lucene.store.IOContext;
|
|||
/**
|
||||
* Lucene 3x implementation of {@link SegmentInfosReader}.
|
||||
* @lucene.experimental
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
|
||||
|
||||
@Override
|
||||
|
@ -56,7 +58,7 @@ public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
|
|||
if (si.getDocStoreIsCompoundFile()) {
|
||||
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
|
||||
si.getDocStoreSegment(), "",
|
||||
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context, false);
|
||||
Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false);
|
||||
}
|
||||
} else if (si.getUseCompoundFile()) {
|
||||
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
|
||||
|
@ -64,7 +66,7 @@ public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
|
|||
}
|
||||
|
||||
try {
|
||||
Lucene40StoredFieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
|
||||
Lucene3xStoredFieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
|
||||
} finally {
|
||||
// If we opened the directory, close it
|
||||
if (dir != directory) dir.close();
|
||||
|
@ -142,7 +144,7 @@ public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
|
|||
if (docStoreOffset != -1) {
|
||||
storesSegment = docStoreSegment;
|
||||
storeIsCompoundFile = docStoreIsCompoundFile;
|
||||
ext = IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
|
||||
ext = Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION;
|
||||
} else {
|
||||
storesSegment = name;
|
||||
storeIsCompoundFile = isCompoundFile;
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
package org.apache.lucene.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.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/** @deprecated */
|
||||
@Deprecated
|
||||
public class Lucene3xStoredFieldsFormat extends StoredFieldsFormat {
|
||||
|
||||
@Override
|
||||
public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si,
|
||||
FieldInfos fn, IOContext context) throws IOException {
|
||||
return new Lucene3xStoredFieldsReader(directory, si, fn, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoredFieldsWriter fieldsWriter(Directory directory, String segment,
|
||||
IOContext context) throws IOException {
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xStoredFieldsReader.files(info, files);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,316 @@
|
|||
package org.apache.lucene.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 org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexFormatTooNewException;
|
||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Class responsible for access to stored document fields.
|
||||
* <p/>
|
||||
* It uses <segment>.fdt and <segment>.fdx; files.
|
||||
*
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
public final class Lucene3xStoredFieldsReader extends StoredFieldsReader implements Cloneable, Closeable {
|
||||
private final static int FORMAT_SIZE = 4;
|
||||
|
||||
/** Extension of stored fields file */
|
||||
public static final String FIELDS_EXTENSION = "fdt";
|
||||
|
||||
/** Extension of stored fields index file */
|
||||
public static final String FIELDS_INDEX_EXTENSION = "fdx";
|
||||
|
||||
// Lucene 3.0: Removal of compressed fields
|
||||
static final int FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS = 2;
|
||||
|
||||
// Lucene 3.2: NumericFields are stored in binary format
|
||||
static final int FORMAT_LUCENE_3_2_NUMERIC_FIELDS = 3;
|
||||
|
||||
// NOTE: if you introduce a new format, make it 1 higher
|
||||
// than the current one, and always change this if you
|
||||
// switch to a new format!
|
||||
public static final int FORMAT_CURRENT = FORMAT_LUCENE_3_2_NUMERIC_FIELDS;
|
||||
|
||||
// when removing support for old versions, leave the last supported version here
|
||||
static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
|
||||
|
||||
// NOTE: bit 0 is free here! You can steal it!
|
||||
public static final int FIELD_IS_BINARY = 1 << 1;
|
||||
|
||||
// the old bit 1 << 2 was compressed, is now left out
|
||||
|
||||
private static final int _NUMERIC_BIT_SHIFT = 3;
|
||||
static final int FIELD_IS_NUMERIC_MASK = 0x07 << _NUMERIC_BIT_SHIFT;
|
||||
|
||||
public static final int FIELD_IS_NUMERIC_INT = 1 << _NUMERIC_BIT_SHIFT;
|
||||
public static final int FIELD_IS_NUMERIC_LONG = 2 << _NUMERIC_BIT_SHIFT;
|
||||
public static final int FIELD_IS_NUMERIC_FLOAT = 3 << _NUMERIC_BIT_SHIFT;
|
||||
public static final int FIELD_IS_NUMERIC_DOUBLE = 4 << _NUMERIC_BIT_SHIFT;
|
||||
|
||||
private final FieldInfos fieldInfos;
|
||||
private final IndexInput fieldsStream;
|
||||
private final IndexInput indexStream;
|
||||
private int numTotalDocs;
|
||||
private int size;
|
||||
private boolean closed;
|
||||
private final int format;
|
||||
|
||||
// The docID offset where our docs begin in the index
|
||||
// file. This will be 0 if we have our own private file.
|
||||
private int docStoreOffset;
|
||||
|
||||
// when we are inside a compound share doc store (CFX),
|
||||
// (lucene 3.0 indexes only), we privately open our own fd.
|
||||
private final CompoundFileDirectory storeCFSReader;
|
||||
|
||||
/** Returns a cloned FieldsReader that shares open
|
||||
* IndexInputs with the original one. It is the caller's
|
||||
* job not to close the original FieldsReader until all
|
||||
* clones are called (eg, currently SegmentReader manages
|
||||
* this logic). */
|
||||
@Override
|
||||
public Lucene3xStoredFieldsReader clone() {
|
||||
ensureOpen();
|
||||
return new Lucene3xStoredFieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, (IndexInput)fieldsStream.clone(), (IndexInput)indexStream.clone());
|
||||
}
|
||||
|
||||
/** 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, "", FIELDS_INDEX_EXTENSION);
|
||||
IndexInput idxStream = dir.openInput(indexStreamFN, IOContext.DEFAULT);
|
||||
|
||||
try {
|
||||
int format = idxStream.readInt();
|
||||
if (format < FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(idxStream, format, FORMAT_MINIMUM, FORMAT_CURRENT);
|
||||
if (format > FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(idxStream, format, FORMAT_MINIMUM, FORMAT_CURRENT);
|
||||
} finally {
|
||||
idxStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
// Used only by clone
|
||||
private Lucene3xStoredFieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, int format, int docStoreOffset,
|
||||
IndexInput fieldsStream, IndexInput indexStream) {
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.numTotalDocs = numTotalDocs;
|
||||
this.size = size;
|
||||
this.format = format;
|
||||
this.docStoreOffset = docStoreOffset;
|
||||
this.fieldsStream = fieldsStream;
|
||||
this.indexStream = indexStream;
|
||||
this.storeCFSReader = null;
|
||||
}
|
||||
|
||||
public Lucene3xStoredFieldsReader(Directory d, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
|
||||
final String segment = si.getDocStoreSegment();
|
||||
final int docStoreOffset = si.getDocStoreOffset();
|
||||
final int size = si.docCount;
|
||||
boolean success = false;
|
||||
fieldInfos = fn;
|
||||
try {
|
||||
if (docStoreOffset != -1 && si.getDocStoreIsCompoundFile()) {
|
||||
d = storeCFSReader = new CompoundFileDirectory(si.dir,
|
||||
IndexFileNames.segmentFileName(segment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false);
|
||||
} else {
|
||||
storeCFSReader = null;
|
||||
}
|
||||
fieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION);
|
||||
indexStream = d.openInput(indexStreamFN, context);
|
||||
|
||||
format = indexStream.readInt();
|
||||
|
||||
if (format < FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(indexStream, format, FORMAT_MINIMUM, FORMAT_CURRENT);
|
||||
if (format > FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(indexStream, format, FORMAT_MINIMUM, FORMAT_CURRENT);
|
||||
|
||||
final long indexSize = indexStream.length() - FORMAT_SIZE;
|
||||
|
||||
if (docStoreOffset != -1) {
|
||||
// We read only a slice out of this shared fields file
|
||||
this.docStoreOffset = docStoreOffset;
|
||||
this.size = size;
|
||||
|
||||
// Verify the file is long enough to hold all of our
|
||||
// docs
|
||||
assert ((int) (indexSize / 8)) >= size + this.docStoreOffset: "indexSize=" + indexSize + " size=" + size + " docStoreOffset=" + docStoreOffset;
|
||||
} else {
|
||||
this.docStoreOffset = 0;
|
||||
this.size = (int) (indexSize >> 3);
|
||||
// Verify two sources of "maxDoc" agree:
|
||||
if (this.size != si.docCount) {
|
||||
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.docCount);
|
||||
}
|
||||
}
|
||||
numTotalDocs = (int) (indexSize >> 3);
|
||||
success = true;
|
||||
} finally {
|
||||
// With lock-less commits, it's entirely possible (and
|
||||
// fine) to hit a FileNotFound exception above. In
|
||||
// this case, we want to explicitly close any subset
|
||||
// of things that were opened so that we don't have to
|
||||
// wait for a GC to do so.
|
||||
if (!success) {
|
||||
close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @throws AlreadyClosedException if this FieldsReader is closed
|
||||
*/
|
||||
private void ensureOpen() throws AlreadyClosedException {
|
||||
if (closed) {
|
||||
throw new AlreadyClosedException("this FieldsReader is closed");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the underlying {@link org.apache.lucene.store.IndexInput} streams.
|
||||
* This means that the Fields values will not be accessible.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
public final void close() throws IOException {
|
||||
if (!closed) {
|
||||
IOUtils.close(fieldsStream, indexStream, storeCFSReader);
|
||||
closed = true;
|
||||
}
|
||||
}
|
||||
|
||||
private void seekIndex(int docID) throws IOException {
|
||||
indexStream.seek(FORMAT_SIZE + (docID + docStoreOffset) * 8L);
|
||||
}
|
||||
|
||||
public final void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
|
||||
seekIndex(n);
|
||||
fieldsStream.seek(indexStream.readLong());
|
||||
|
||||
final int numFields = fieldsStream.readVInt();
|
||||
for (int fieldIDX = 0; fieldIDX < numFields; fieldIDX++) {
|
||||
int fieldNumber = fieldsStream.readVInt();
|
||||
FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
|
||||
|
||||
int bits = fieldsStream.readByte() & 0xFF;
|
||||
assert bits <= (FIELD_IS_NUMERIC_MASK | FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits);
|
||||
|
||||
switch(visitor.needsField(fieldInfo)) {
|
||||
case YES:
|
||||
readField(visitor, fieldInfo, bits);
|
||||
break;
|
||||
case NO:
|
||||
skipField(bits);
|
||||
break;
|
||||
case STOP:
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void readField(StoredFieldVisitor visitor, FieldInfo info, int bits) throws IOException {
|
||||
final int numeric = bits & FIELD_IS_NUMERIC_MASK;
|
||||
if (numeric != 0) {
|
||||
switch(numeric) {
|
||||
case FIELD_IS_NUMERIC_INT:
|
||||
visitor.intField(info, fieldsStream.readInt());
|
||||
return;
|
||||
case FIELD_IS_NUMERIC_LONG:
|
||||
visitor.longField(info, fieldsStream.readLong());
|
||||
return;
|
||||
case FIELD_IS_NUMERIC_FLOAT:
|
||||
visitor.floatField(info, Float.intBitsToFloat(fieldsStream.readInt()));
|
||||
return;
|
||||
case FIELD_IS_NUMERIC_DOUBLE:
|
||||
visitor.doubleField(info, Double.longBitsToDouble(fieldsStream.readLong()));
|
||||
return;
|
||||
default:
|
||||
throw new CorruptIndexException("Invalid numeric type: " + Integer.toHexString(numeric));
|
||||
}
|
||||
} else {
|
||||
final int length = fieldsStream.readVInt();
|
||||
byte bytes[] = new byte[length];
|
||||
fieldsStream.readBytes(bytes, 0, length);
|
||||
if ((bits & FIELD_IS_BINARY) != 0) {
|
||||
visitor.binaryField(info, bytes, 0, bytes.length);
|
||||
} else {
|
||||
visitor.stringField(info, new String(bytes, 0, bytes.length, IOUtils.CHARSET_UTF_8));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void skipField(int bits) throws IOException {
|
||||
final int numeric = bits & FIELD_IS_NUMERIC_MASK;
|
||||
if (numeric != 0) {
|
||||
switch(numeric) {
|
||||
case FIELD_IS_NUMERIC_INT:
|
||||
case FIELD_IS_NUMERIC_FLOAT:
|
||||
fieldsStream.readInt();
|
||||
return;
|
||||
case FIELD_IS_NUMERIC_LONG:
|
||||
case FIELD_IS_NUMERIC_DOUBLE:
|
||||
fieldsStream.readLong();
|
||||
return;
|
||||
default:
|
||||
throw new CorruptIndexException("Invalid numeric type: " + Integer.toHexString(numeric));
|
||||
}
|
||||
} else {
|
||||
final int length = fieldsStream.readVInt();
|
||||
fieldsStream.seek(fieldsStream.getFilePointer() + length);
|
||||
}
|
||||
}
|
||||
|
||||
// note: if there are shared docstores, we are also called by Lucene3xCodec even in
|
||||
// the CFS case. so logic here must handle this.
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
assert info.getDocStoreSegment() != null;
|
||||
if (info.getDocStoreIsCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION));
|
||||
} else {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", FIELDS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", FIELDS_EXTENSION));
|
||||
}
|
||||
} else if (!info.getUseCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", FIELDS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", FIELDS_EXTENSION));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -48,8 +48,8 @@ public class Lucene3xTermVectorsFormat extends TermVectorsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xTermVectorsReader.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xTermVectorsReader.files(info, files);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.index.IndexFormatTooOldException;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -45,6 +46,8 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/** @deprecated */
|
||||
@Deprecated
|
||||
public class Lucene3xTermVectorsReader extends TermVectorsReader {
|
||||
|
||||
// NOTE: if you make a new format, it must be larger than
|
||||
|
@ -88,6 +91,12 @@ public class Lucene3xTermVectorsReader extends TermVectorsReader {
|
|||
// file. This will be 0 if we have our own private file.
|
||||
private int docStoreOffset;
|
||||
|
||||
// when we are inside a compound share doc store (CFX),
|
||||
// (lucene 3.0 indexes only), we privately open our own fd.
|
||||
// TODO: if we are worried, maybe we could eliminate the
|
||||
// extra fd somehow when you also have vectors...
|
||||
private final CompoundFileDirectory storeCFSReader;
|
||||
|
||||
private final int format;
|
||||
|
||||
// used by clone
|
||||
|
@ -100,6 +109,7 @@ public class Lucene3xTermVectorsReader extends TermVectorsReader {
|
|||
this.numTotalDocs = numTotalDocs;
|
||||
this.docStoreOffset = docStoreOffset;
|
||||
this.format = format;
|
||||
this.storeCFSReader = null;
|
||||
}
|
||||
|
||||
public Lucene3xTermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context)
|
||||
|
@ -111,6 +121,12 @@ public class Lucene3xTermVectorsReader extends TermVectorsReader {
|
|||
boolean success = false;
|
||||
|
||||
try {
|
||||
if (docStoreOffset != -1 && si.getDocStoreIsCompoundFile()) {
|
||||
d = storeCFSReader = new CompoundFileDirectory(si.dir,
|
||||
IndexFileNames.segmentFileName(segment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false);
|
||||
} else {
|
||||
storeCFSReader = null;
|
||||
}
|
||||
String idxName = IndexFileNames.segmentFileName(segment, "", VECTORS_INDEX_EXTENSION);
|
||||
tvx = d.openInput(idxName, context);
|
||||
format = checkValidFormat(tvx);
|
||||
|
@ -168,7 +184,7 @@ public class Lucene3xTermVectorsReader extends TermVectorsReader {
|
|||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.close(tvx, tvd, tvf);
|
||||
IOUtils.close(tvx, tvd, tvf, storeCFSReader);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -674,16 +690,20 @@ public class Lucene3xTermVectorsReader extends TermVectorsReader {
|
|||
return new Lucene3xTermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, docStoreOffset, format);
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
// note: if there are shared docstores, we are also called by Lucene3xCodec even in
|
||||
// the CFS case. so logic here must handle this.
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getHasVectors()) {
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
assert info.getDocStoreSegment() != null;
|
||||
if (!info.getDocStoreIsCompoundFile()) {
|
||||
if (info.getDocStoreIsCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION));
|
||||
} else {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_FIELDS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_DOCUMENTS_EXTENSION));
|
||||
}
|
||||
} else {
|
||||
} else if (!info.getUseCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
|
||||
|
|
|
@ -36,7 +36,9 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* index segment. Pairs are accessed either by Term or by ordinal position the
|
||||
* set. The Terms and TermInfo are actually serialized and stored into a byte
|
||||
* array and pointers to the position of each are stored in a int array.
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
class TermInfosReaderIndex {
|
||||
|
||||
private static final int MAX_PAGE_BITS = 18; // 256 KB block
|
||||
|
|
|
@ -66,14 +66,14 @@ public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
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));
|
||||
assert segmentInfo.dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
assert segmentInfo.dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.codecs.PerDocProducer;
|
|||
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 Lucene40DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
|
@ -41,7 +40,7 @@ public class Lucene40DocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40DocValuesConsumer.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40DocValuesConsumer.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.lucene.codecs.FieldInfosFormat;
|
|||
import org.apache.lucene.codecs.FieldInfosReader;
|
||||
import org.apache.lucene.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
|
@ -44,7 +43,7 @@ public class Lucene40FieldInfosFormat extends FieldInfosFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40FieldInfosReader.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40FieldInfosReader.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -145,7 +145,7 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene40FieldInfosWriter.FIELD_INFOS_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,7 +48,7 @@ public class Lucene40LiveDocsFormat extends LiveDocsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.hasDeletions()) {
|
||||
files.add(IndexFileNames.fileNameFromGeneration(info.name, DELETES_EXTENSION, info.getDelGen()));
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.lucene.index.IndexReader;
|
|||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Norms Format for the default codec.
|
||||
|
@ -50,10 +49,8 @@ public class Lucene40NormsFormat extends NormsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files)
|
||||
throws IOException {
|
||||
Lucene40NormsDocValuesConsumer.files(dir, info, files);
|
||||
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40NormsDocValuesConsumer.files(info, files);
|
||||
}
|
||||
|
||||
public static class Lucene40NormsDocValuesProducer extends Lucene40DocValuesProducer {
|
||||
|
@ -103,7 +100,7 @@ public class Lucene40NormsFormat extends NormsFormat {
|
|||
return info.getNormType();
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
final String normsFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION);
|
||||
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.codecs.PostingsWriterBase;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Provides a {@link PostingsReaderBase} and {@link
|
||||
|
@ -52,7 +51,7 @@ public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.lucene.codecs.PostingsWriterBase;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** Default codec.
|
||||
* @lucene.experimental */
|
||||
|
@ -106,9 +105,9 @@ public class Lucene40PostingsFormat extends PostingsFormat {
|
|||
static final String PROX_EXTENSION = "prx";
|
||||
|
||||
@Override
|
||||
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);
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -74,7 +74,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) throws IOException {
|
||||
public static void files(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, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION));
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.Map;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.SegmentInfosReader;
|
||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentInfos;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
|
|
|
@ -44,7 +44,7 @@ public class Lucene40StoredFieldsFormat extends StoredFieldsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40StoredFieldsReader.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40StoredFieldsReader.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,8 +24,6 @@ import org.apache.lucene.index.CorruptIndexException;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexFormatTooNewException;
|
||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
|
@ -35,7 +33,6 @@ import org.apache.lucene.store.IndexInput;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -54,11 +51,6 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
private int numTotalDocs;
|
||||
private int size;
|
||||
private boolean closed;
|
||||
private final int format;
|
||||
|
||||
// The docID offset where our docs begin in the index
|
||||
// file. This will be 0 if we have our own private file.
|
||||
private int docStoreOffset;
|
||||
|
||||
/** Returns a cloned FieldsReader that shares open
|
||||
* IndexInputs with the original one. It is the caller's
|
||||
|
@ -68,41 +60,20 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
@Override
|
||||
public Lucene40StoredFieldsReader clone() {
|
||||
ensureOpen();
|
||||
return new Lucene40StoredFieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, (IndexInput)fieldsStream.clone(), (IndexInput)indexStream.clone());
|
||||
}
|
||||
|
||||
/** 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, "", Lucene40StoredFieldsWriter.FIELDS_INDEX_EXTENSION);
|
||||
IndexInput idxStream = dir.openInput(indexStreamFN, IOContext.DEFAULT);
|
||||
|
||||
try {
|
||||
int format = idxStream.readInt();
|
||||
if (format < Lucene40StoredFieldsWriter.FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(idxStream, format, Lucene40StoredFieldsWriter.FORMAT_MINIMUM, Lucene40StoredFieldsWriter.FORMAT_CURRENT);
|
||||
if (format > Lucene40StoredFieldsWriter.FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(idxStream, format, Lucene40StoredFieldsWriter.FORMAT_MINIMUM, Lucene40StoredFieldsWriter.FORMAT_CURRENT);
|
||||
} finally {
|
||||
idxStream.close();
|
||||
}
|
||||
return new Lucene40StoredFieldsReader(fieldInfos, numTotalDocs, size, (IndexInput)fieldsStream.clone(), (IndexInput)indexStream.clone());
|
||||
}
|
||||
|
||||
// Used only by clone
|
||||
private Lucene40StoredFieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, int format, int docStoreOffset,
|
||||
IndexInput fieldsStream, IndexInput indexStream) {
|
||||
private Lucene40StoredFieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, IndexInput fieldsStream, IndexInput indexStream) {
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.numTotalDocs = numTotalDocs;
|
||||
this.size = size;
|
||||
this.format = format;
|
||||
this.docStoreOffset = docStoreOffset;
|
||||
this.fieldsStream = fieldsStream;
|
||||
this.indexStream = indexStream;
|
||||
}
|
||||
|
||||
public Lucene40StoredFieldsReader(Directory d, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
|
||||
final String segment = si.getDocStoreSegment();
|
||||
final int docStoreOffset = si.getDocStoreOffset();
|
||||
final int size = si.docCount;
|
||||
final String segment = si.name;
|
||||
boolean success = false;
|
||||
fieldInfos = fn;
|
||||
try {
|
||||
|
@ -110,30 +81,17 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", Lucene40StoredFieldsWriter.FIELDS_INDEX_EXTENSION);
|
||||
indexStream = d.openInput(indexStreamFN, context);
|
||||
|
||||
format = indexStream.readInt();
|
||||
|
||||
if (format < Lucene40StoredFieldsWriter.FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(indexStream, format, Lucene40StoredFieldsWriter.FORMAT_MINIMUM, Lucene40StoredFieldsWriter.FORMAT_CURRENT);
|
||||
if (format > Lucene40StoredFieldsWriter.FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(indexStream, format, Lucene40StoredFieldsWriter.FORMAT_MINIMUM, Lucene40StoredFieldsWriter.FORMAT_CURRENT);
|
||||
// its a 4.0 codec: so its not too-old, its corrupt.
|
||||
// TODO: change this to CodecUtil.checkHeader
|
||||
if (Lucene40StoredFieldsWriter.FORMAT_CURRENT != indexStream.readInt()) {
|
||||
throw new CorruptIndexException("unexpected fdx header: " + indexStream);
|
||||
}
|
||||
|
||||
final long indexSize = indexStream.length() - FORMAT_SIZE;
|
||||
|
||||
if (docStoreOffset != -1) {
|
||||
// We read only a slice out of this shared fields file
|
||||
this.docStoreOffset = docStoreOffset;
|
||||
this.size = size;
|
||||
|
||||
// Verify the file is long enough to hold all of our
|
||||
// docs
|
||||
assert ((int) (indexSize / 8)) >= size + this.docStoreOffset: "indexSize=" + indexSize + " size=" + size + " docStoreOffset=" + docStoreOffset;
|
||||
} else {
|
||||
this.docStoreOffset = 0;
|
||||
this.size = (int) (indexSize >> 3);
|
||||
// Verify two sources of "maxDoc" agree:
|
||||
if (this.size != si.docCount) {
|
||||
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.docCount);
|
||||
}
|
||||
this.size = (int) (indexSize >> 3);
|
||||
// Verify two sources of "maxDoc" agree:
|
||||
if (this.size != si.docCount) {
|
||||
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.docCount);
|
||||
}
|
||||
numTotalDocs = (int) (indexSize >> 3);
|
||||
success = true;
|
||||
|
@ -176,7 +134,7 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
}
|
||||
|
||||
private void seekIndex(int docID) throws IOException {
|
||||
indexStream.seek(FORMAT_SIZE + (docID + docStoreOffset) * 8L);
|
||||
indexStream.seek(FORMAT_SIZE + docID * 8L);
|
||||
}
|
||||
|
||||
public final void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
|
||||
|
@ -203,8 +161,6 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
static final Charset UTF8 = Charset.forName("UTF-8");
|
||||
|
||||
private void readField(StoredFieldVisitor visitor, FieldInfo info, int bits) throws IOException {
|
||||
final int numeric = bits & Lucene40StoredFieldsWriter.FIELD_IS_NUMERIC_MASK;
|
||||
|
@ -232,7 +188,7 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
if ((bits & Lucene40StoredFieldsWriter.FIELD_IS_BINARY) != 0) {
|
||||
visitor.binaryField(info, bytes, 0, bytes.length);
|
||||
} else {
|
||||
visitor.stringField(info, new String(bytes, 0, bytes.length, UTF8));
|
||||
visitor.stringField(info, new String(bytes, 0, bytes.length, IOUtils.CHARSET_UTF_8));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -269,7 +225,7 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
int count = 0;
|
||||
while (count < numDocs) {
|
||||
final long offset;
|
||||
final int docID = docStoreOffset + startDocID + count + 1;
|
||||
final int docID = startDocID + count + 1;
|
||||
assert docID <= numTotalDocs;
|
||||
if (docID < numTotalDocs)
|
||||
offset = indexStream.readLong();
|
||||
|
@ -283,19 +239,9 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
|
||||
return fieldsStream;
|
||||
}
|
||||
|
||||
// TODO: split into PreFlexFieldsReader so it can handle this shared docstore crap?
|
||||
// only preflex segments refer to these?
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
assert info.getDocStoreSegment() != null;
|
||||
if (!info.getDocStoreIsCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", Lucene40StoredFieldsWriter.FIELDS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", Lucene40StoredFieldsWriter.FIELDS_EXTENSION));
|
||||
}
|
||||
} else {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene40StoredFieldsWriter.FIELDS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene40StoredFieldsWriter.FIELDS_EXTENSION));
|
||||
}
|
||||
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene40StoredFieldsWriter.FIELDS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene40StoredFieldsWriter.FIELDS_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,10 +55,7 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
|
|||
// currently unused: static final int FIELD_IS_NUMERIC_SHORT = 5 << _NUMERIC_BIT_SHIFT;
|
||||
// currently unused: static final int FIELD_IS_NUMERIC_BYTE = 6 << _NUMERIC_BIT_SHIFT;
|
||||
|
||||
// Lucene 3.0: Removal of compressed fields
|
||||
static final int FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS = 2;
|
||||
|
||||
// Lucene 3.2: NumericFields are stored in binary format
|
||||
// (Happens to be the same as for now) Lucene 3.2: NumericFields are stored in binary format
|
||||
static final int FORMAT_LUCENE_3_2_NUMERIC_FIELDS = 3;
|
||||
|
||||
// NOTE: if you introduce a new format, make it 1 higher
|
||||
|
@ -67,7 +64,7 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
|
|||
static final int FORMAT_CURRENT = FORMAT_LUCENE_3_2_NUMERIC_FIELDS;
|
||||
|
||||
// when removing support for old versions, leave the last supported version here
|
||||
static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
|
||||
static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_2_NUMERIC_FIELDS;
|
||||
|
||||
/** Extension of stored fields file */
|
||||
public static final String FIELDS_EXTENSION = "fdt";
|
||||
|
@ -211,7 +208,7 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
|
|||
// throw an exception to prevent the corruption from
|
||||
// entering the index. See LUCENE-1282 for
|
||||
// details.
|
||||
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + numDocs + " but fdx file size is " + indexStream.getFilePointer() + " file=" + indexStream.toString() + "; now aborting this merge to prevent index corruption");
|
||||
throw new RuntimeException("fdx size mismatch: docCount is " + numDocs + " but fdx file size is " + indexStream.getFilePointer() + " file=" + indexStream.toString() + "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -41,7 +41,7 @@ public class Lucene40TermVectorsFormat extends TermVectorsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40TermVectorsReader.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40TermVectorsReader.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -699,7 +699,7 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
|
|||
return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, format);
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getHasVectors()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
|
||||
|
|
|
@ -355,7 +355,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
|
|||
// throw an exception to prevent the corruption from
|
||||
// entering the index. See LUCENE-1282 for
|
||||
// details.
|
||||
throw new RuntimeException("mergeVectors produced an invalid result: mergedDocs is " + numDocs + " but tvx size is " + tvx.getFilePointer() + " file=" + tvx.toString() + "; now aborting this merge to prevent index corruption");
|
||||
throw new RuntimeException("tvx size mismatch: mergedDocs is " + numDocs + " but tvx size is " + tvx.getFilePointer() + " file=" + tvx.toString() + "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
|
||||
/** Close all streams. */
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
@ -826,7 +825,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -306,8 +306,8 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(final Directory dir, final SegmentInfo info, String segmentSuffix, final Set<String> files)
|
||||
throws IOException {
|
||||
public void files(final SegmentInfo info, String segmentSuffix, final Set<String> files) throws IOException {
|
||||
final Directory dir = info.dir;
|
||||
|
||||
final String mapFileName = IndexFileNames.segmentFileName(info.name, segmentSuffix, PER_FIELD_EXTENSION);
|
||||
files.add(mapFileName);
|
||||
|
@ -316,7 +316,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
|||
new VisitPerFieldFile(dir, info.name, segmentSuffix) {
|
||||
@Override
|
||||
protected void visitOneFormat(String segmentSuffix, PostingsFormat format) throws IOException {
|
||||
format.files(dir, info, segmentSuffix, files);
|
||||
format.files(info, segmentSuffix, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.lucene.codecs.PostingsWriterBase;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** This postings format "inlines" the postings for terms that have
|
||||
* low docFreq. It wraps another postings format, which is used for
|
||||
|
@ -115,8 +114,8 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
wrappedPostingsBaseFormat.files(dir, segmentInfo, segmentSuffix, files);
|
||||
BlockTreeTermsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
wrappedPostingsBaseFormat.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.util.IOUtils;
|
|||
public class SepDocValuesConsumer extends DocValuesWriterBase {
|
||||
private final Directory directory;
|
||||
private final FieldInfos fieldInfos;
|
||||
|
||||
public SepDocValuesConsumer(PerDocWriteState state) throws IOException {
|
||||
super(state);
|
||||
this.directory = state.directory;
|
||||
|
@ -49,9 +50,9 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
|
|||
return directory;
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo,
|
||||
public static void files(SegmentInfo segmentInfo,
|
||||
Set<String> files) throws IOException {
|
||||
files(dir, segmentInfo.getFieldInfos(), segmentInfo.name, files);
|
||||
files(segmentInfo.dir, segmentInfo.getFieldInfos(), segmentInfo.name, files);
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough")
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.lucene.codecs.FieldInfosFormat;
|
|||
import org.apache.lucene.codecs.FieldInfosReader;
|
||||
import org.apache.lucene.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* plaintext field infos format
|
||||
|
@ -47,7 +46,7 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextFieldInfosReader.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextFieldInfosReader.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -133,7 +133,7 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
|
|||
return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", FIELD_INFOS_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -138,7 +138,7 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.hasDeletions()) {
|
||||
files.add(IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen()));
|
||||
}
|
||||
|
|
|
@ -280,7 +280,7 @@ public class SimpleTextNormsConsumer extends PerDocConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = info.getFieldInfos();
|
||||
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.codecs.PerDocProducer;
|
|||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* plain-text norms format
|
||||
|
@ -47,9 +46,7 @@ public class SimpleTextNormsFormat extends NormsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files)
|
||||
throws IOException {
|
||||
SimpleTextNormsConsumer.files(dir, info, files);
|
||||
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextNormsConsumer.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.index.SegmentInfo;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** For debugging, curiosity, transparency only!! Do not
|
||||
* use this codec in production.
|
||||
|
@ -61,7 +60,7 @@ public class SimpleTextPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
files.add(getPostingsFileName(segmentInfo.name, segmentSuffix));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,7 +47,7 @@ public class SimpleTextStoredFieldsFormat extends StoredFieldsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextStoredFieldsReader.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextStoredFieldsReader.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -178,7 +178,7 @@ public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", SimpleTextStoredFieldsWriter.FIELDS_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -47,7 +47,7 @@ public class SimpleTextTermVectorsFormat extends TermVectorsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextTermVectorsReader.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextTermVectorsReader.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -201,7 +201,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getHasVectors()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_EXTENSION));
|
||||
}
|
||||
|
|
|
@ -494,14 +494,11 @@ public class DocumentsWriterPerThread {
|
|||
}
|
||||
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
final double newSegmentSizeNoStore = newSegment.sizeInBytes(false)/1024./1024.;
|
||||
final double newSegmentSize = newSegment.sizeInBytes(true)/1024./1024.;
|
||||
final double newSegmentSize = newSegment.sizeInBytes()/1024./1024.;
|
||||
infoStream.message("DWPT", "flushed: segment=" + newSegment +
|
||||
" ramUsed=" + nf.format(startMBUsed) + " MB" +
|
||||
" newFlushedSize=" + nf.format(newSegmentSize) + " MB" +
|
||||
" (" + nf.format(newSegmentSizeNoStore) + " MB w/o doc stores)" +
|
||||
" docs/MB=" + nf.format(flushedDocCount / newSegmentSize) +
|
||||
" new/old=" + nf.format(100.0 * newSegmentSizeNoStore / startMBUsed) + "%");
|
||||
" newFlushedSize(includes docstores)=" + nf.format(newSegmentSize) + " MB" +
|
||||
" docs/MB=" + nf.format(flushedDocCount / newSegmentSize));
|
||||
}
|
||||
doAfterFlush();
|
||||
success = true;
|
||||
|
|
|
@ -54,9 +54,6 @@ public final class IndexFileNames {
|
|||
/** Extension of compound file entries */
|
||||
public static final String COMPOUND_FILE_ENTRIES_EXTENSION = "cfe";
|
||||
|
||||
/** Extension of compound file for doc store files*/
|
||||
public static final String COMPOUND_FILE_STORE_EXTENSION = "cfx";
|
||||
|
||||
/**
|
||||
* This array contains all filename extensions used by
|
||||
* Lucene's index files, with one exception, namely the
|
||||
|
@ -68,7 +65,6 @@ public final class IndexFileNames {
|
|||
COMPOUND_FILE_EXTENSION,
|
||||
COMPOUND_FILE_ENTRIES_EXTENSION,
|
||||
GEN_EXTENSION,
|
||||
COMPOUND_FILE_STORE_EXTENSION,
|
||||
};
|
||||
|
||||
/**
|
||||
|
|
|
@ -2545,27 +2545,27 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// only relevant for segments that share doc store with others,
|
||||
// because the DS might have been copied already, in which case we
|
||||
// just want to update the DS name of this SegmentInfo.
|
||||
// NOTE: pre-3x segments include a null DSName if they don't share doc
|
||||
// store. The following code ensures we don't accidentally insert
|
||||
// 'null' to the map.
|
||||
String dsName = info.getDocStoreSegment();
|
||||
assert dsName != null;
|
||||
final String newDsName;
|
||||
if (dsName != null) {
|
||||
if (dsNames.containsKey(dsName)) {
|
||||
newDsName = dsNames.get(dsName);
|
||||
} else {
|
||||
dsNames.put(dsName, segName);
|
||||
newDsName = segName;
|
||||
}
|
||||
if (dsNames.containsKey(dsName)) {
|
||||
newDsName = dsNames.get(dsName);
|
||||
} else {
|
||||
dsNames.put(dsName, segName);
|
||||
newDsName = segName;
|
||||
}
|
||||
|
||||
Set<String> codecDocStoreFiles = info.codecDocStoreFiles();
|
||||
Set<String> codecDocStoreFiles = new HashSet<String>();
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
// only violate the codec this way if its preflex
|
||||
codec.storedFieldsFormat().files(info, codecDocStoreFiles);
|
||||
codec.termVectorsFormat().files(info, codecDocStoreFiles);
|
||||
}
|
||||
|
||||
// Copy the segment files
|
||||
for (String file: info.files()) {
|
||||
final String newFileName;
|
||||
if (codecDocStoreFiles.contains(file) || file.endsWith(IndexFileNames.COMPOUND_FILE_STORE_EXTENSION)) {
|
||||
if (codecDocStoreFiles.contains(file)) {
|
||||
newFileName = newDsName + IndexFileNames.stripSegmentName(file);
|
||||
if (dsFilesCopied.contains(newFileName)) {
|
||||
continue;
|
||||
|
@ -4070,12 +4070,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
*/
|
||||
static final Collection<String> createCompoundFile(Directory directory, String fileName, CheckAbort checkAbort, final SegmentInfo info, IOContext context)
|
||||
throws IOException {
|
||||
|
||||
assert info.getDocStoreOffset() == -1;
|
||||
// Now merge all added files
|
||||
Collection<String> files = info.files();
|
||||
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
|
||||
try {
|
||||
assert assertNoSeparateFiles(files, directory, info);
|
||||
for (String file : files) {
|
||||
directory.copy(cfsDir, file, file, context);
|
||||
checkAbort.work(directory.fileLength(file));
|
||||
|
@ -4086,20 +4085,4 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
return files;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* used only by assert: checks that filenames about to be put in cfs belong.
|
||||
*/
|
||||
private static boolean assertNoSeparateFiles(Collection<String> files,
|
||||
Directory dir, SegmentInfo info) throws IOException {
|
||||
// maybe this is overkill, but codec naming clashes would be bad.
|
||||
Set<String> separateFiles = new HashSet<String>();
|
||||
info.getCodec().separateFiles(dir, info, separateFiles);
|
||||
|
||||
for (String file : files) {
|
||||
assert !separateFiles.contains(file) : file + " should not go in CFS!";
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,16 +48,12 @@ final class SegmentCoreReaders {
|
|||
// SegmentReaders:
|
||||
private final AtomicInteger ref = new AtomicInteger(1);
|
||||
|
||||
final String segment;
|
||||
final FieldInfos fieldInfos;
|
||||
|
||||
final FieldsProducer fields;
|
||||
final PerDocProducer perDocProducer;
|
||||
final PerDocProducer norms;
|
||||
|
||||
final Directory dir;
|
||||
final Directory cfsDir;
|
||||
final IOContext context;
|
||||
final int termsIndexDivisor;
|
||||
|
||||
private final SegmentReader owner;
|
||||
|
@ -65,7 +61,6 @@ final class SegmentCoreReaders {
|
|||
final StoredFieldsReader fieldsReaderOrig;
|
||||
final TermVectorsReader termVectorsReaderOrig;
|
||||
final CompoundFileDirectory cfsReader;
|
||||
final CompoundFileDirectory storeCFSReader;
|
||||
|
||||
final CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new CloseableThreadLocal<StoredFieldsReader>() {
|
||||
@Override
|
||||
|
@ -91,22 +86,18 @@ final class SegmentCoreReaders {
|
|||
throw new IllegalArgumentException("indexDivisor must be < 0 (don't load terms index) or greater than 0 (got 0)");
|
||||
}
|
||||
|
||||
segment = si.name;
|
||||
final Codec codec = si.getCodec();
|
||||
this.context = context;
|
||||
this.dir = dir;
|
||||
|
||||
final Directory cfsDir; // confusing name: if (cfs) its the cfsdir, otherwise its the segment's directory.
|
||||
|
||||
boolean success = false;
|
||||
|
||||
try {
|
||||
Directory dir0 = dir;
|
||||
if (si.getUseCompoundFile()) {
|
||||
cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
|
||||
dir0 = cfsReader;
|
||||
cfsDir = cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
|
||||
} else {
|
||||
cfsReader = null;
|
||||
cfsDir = dir;
|
||||
}
|
||||
cfsDir = dir0;
|
||||
si.loadFieldInfos(cfsDir, false); // prevent opening the CFS to load fieldInfos
|
||||
fieldInfos = si.getFieldInfos();
|
||||
|
||||
|
@ -119,36 +110,13 @@ final class SegmentCoreReaders {
|
|||
// ask codec for its Norms:
|
||||
// TODO: since we don't write any norms file if there are no norms,
|
||||
// kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
|
||||
norms = codec.normsFormat().docsProducer(segmentReadState, dir);
|
||||
norms = codec.normsFormat().docsProducer(segmentReadState);
|
||||
perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
|
||||
|
||||
final Directory storeDir;
|
||||
if (si.getDocStoreOffset() != -1) {
|
||||
if (si.getDocStoreIsCompoundFile()) {
|
||||
storeCFSReader = new CompoundFileDirectory(dir,
|
||||
IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
|
||||
context, false);
|
||||
storeDir = storeCFSReader;
|
||||
assert storeDir != null;
|
||||
} else {
|
||||
storeCFSReader = null;
|
||||
storeDir = dir;
|
||||
assert storeDir != null;
|
||||
}
|
||||
} else if (si.getUseCompoundFile()) {
|
||||
storeDir = cfsReader;
|
||||
storeCFSReader = null;
|
||||
assert storeDir != null;
|
||||
} else {
|
||||
storeDir = dir;
|
||||
storeCFSReader = null;
|
||||
assert storeDir != null;
|
||||
}
|
||||
|
||||
fieldsReaderOrig = si.getCodec().storedFieldsFormat().fieldsReader(storeDir, si, fieldInfos, context);
|
||||
|
||||
fieldsReaderOrig = si.getCodec().storedFieldsFormat().fieldsReader(cfsDir, si, fieldInfos, context);
|
||||
|
||||
if (si.getHasVectors()) { // open term vector files only as needed
|
||||
termVectorsReaderOrig = si.getCodec().termVectorsFormat().vectorsReader(storeDir, si, fieldInfos, context);
|
||||
termVectorsReaderOrig = si.getCodec().termVectorsFormat().vectorsReader(cfsDir, si, fieldInfos, context);
|
||||
} else {
|
||||
termVectorsReaderOrig = null;
|
||||
}
|
||||
|
@ -175,7 +143,7 @@ final class SegmentCoreReaders {
|
|||
//System.out.println("core.decRef seg=" + owner.getSegmentInfo() + " rc=" + ref);
|
||||
if (ref.decrementAndGet() == 0) {
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, perDocProducer,
|
||||
termVectorsReaderOrig, fieldsReaderOrig, cfsReader, storeCFSReader, norms);
|
||||
termVectorsReaderOrig, fieldsReaderOrig, cfsReader, norms);
|
||||
notifyCoreClosedListeners();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.lucene.store.CompoundFileDirectory;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* Information about a segment such as it's name, directory, and files related
|
||||
|
@ -74,8 +73,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
private volatile List<String> files; // cached list of files that this segment uses
|
||||
// in the Directory
|
||||
|
||||
private volatile long sizeInBytesNoStore = -1; // total byte size of all but the store files (computed on demand)
|
||||
private volatile long sizeInBytesWithStore = -1; // total byte size of all of our files (computed on demand)
|
||||
private volatile long sizeInBytes = -1; // total byte size of all files (computed on demand)
|
||||
|
||||
//TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0)
|
||||
private int docStoreOffset; // if this segment shares stored fields & vectors, this
|
||||
|
@ -213,63 +211,12 @@ public final class SegmentInfo implements Cloneable {
|
|||
* Returns total size in bytes of all of files used by this segment
|
||||
*/
|
||||
public long sizeInBytes() throws IOException {
|
||||
return sizeInBytes(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns total size in bytes of all of files used by this segment (if
|
||||
* {@code includeDocStores} is true), or the size of all files except the
|
||||
* store files otherwise.
|
||||
* <p>
|
||||
* NOTE: includeDocStores=false should only be used for debugging.
|
||||
* Theoretically a codec could combine its files however it wants (after-
|
||||
* the-fact or something), and this calculation is not particularly
|
||||
* efficient.
|
||||
*/
|
||||
long sizeInBytes(boolean includeDocStores) throws IOException {
|
||||
// TODO: based on how this is used, can't we just forget about all this docstore crap?
|
||||
// its really an abstraction violation into the codec
|
||||
if (includeDocStores) {
|
||||
if (sizeInBytesWithStore != -1) {
|
||||
return sizeInBytesWithStore;
|
||||
}
|
||||
long sum = 0;
|
||||
for (final String fileName : files()) {
|
||||
// We don't count bytes used by a shared doc store
|
||||
// against this segment
|
||||
if (docStoreOffset == -1 || !isDocStoreFile(fileName)) {
|
||||
sum += dir.fileLength(fileName);
|
||||
}
|
||||
}
|
||||
sizeInBytesWithStore = sum;
|
||||
return sizeInBytesWithStore;
|
||||
} else {
|
||||
if (sizeInBytesNoStore != -1) {
|
||||
return sizeInBytesNoStore;
|
||||
}
|
||||
long sum = 0;
|
||||
for (final String fileName : files()) {
|
||||
if (isDocStoreFile(fileName)) {
|
||||
continue;
|
||||
}
|
||||
sum += dir.fileLength(fileName);
|
||||
}
|
||||
sizeInBytesNoStore = sum;
|
||||
return sizeInBytesNoStore;
|
||||
}
|
||||
}
|
||||
|
||||
Set<String> codecDocStoreFiles() throws IOException {
|
||||
Set<String> docStoreFiles = new HashSet<String>();
|
||||
codec.storedFieldsFormat().files(dir, this, docStoreFiles);
|
||||
codec.termVectorsFormat().files(dir, this, docStoreFiles);
|
||||
return docStoreFiles;
|
||||
}
|
||||
|
||||
// TODO: a little messy, but sizeInBytes above that uses this is the real problem.
|
||||
private boolean isDocStoreFile(String fileName) throws IOException {
|
||||
Set<String> docStoreFiles = codecDocStoreFiles();
|
||||
return fileName.endsWith(IndexFileNames.COMPOUND_FILE_STORE_EXTENSION) || docStoreFiles.contains(fileName);
|
||||
sizeInBytes = sum;
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
public boolean getHasVectors() throws IOException {
|
||||
|
@ -473,30 +420,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
final Set<String> fileSet = new HashSet<String>();
|
||||
|
||||
boolean useCompoundFile = getUseCompoundFile();
|
||||
|
||||
if (useCompoundFile) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
if (version != null && StringHelper.getVersionComparator().compare("4.0", version) <= 0) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "",
|
||||
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
}
|
||||
} else {
|
||||
codec.files(dir, this, fileSet);
|
||||
}
|
||||
|
||||
// regardless of compound file setting: these files are always in the directory
|
||||
codec.separateFiles(dir, this, fileSet);
|
||||
|
||||
if (docStoreOffset != -1) {
|
||||
// We are sharing doc stores (stored fields, term
|
||||
// vectors) with other segments
|
||||
assert docStoreSegment != null;
|
||||
// TODO: push this out into preflex fieldsFormat?
|
||||
if (docStoreIsCompoundFile) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION));
|
||||
}
|
||||
}
|
||||
codec.files(this, fileSet);
|
||||
|
||||
files = new ArrayList<String>(fileSet);
|
||||
|
||||
|
@ -507,8 +431,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
* files this segment has. */
|
||||
private void clearFilesCache() {
|
||||
files = null;
|
||||
sizeInBytesNoStore = -1;
|
||||
sizeInBytesWithStore = -1;
|
||||
sizeInBytes = -1;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
|
|
|
@ -183,7 +183,7 @@ public final class SegmentReader extends IndexReader {
|
|||
public String toString() {
|
||||
// SegmentInfo.toString takes dir and number of
|
||||
// *pending* deletions; so we reverse compute that here:
|
||||
return si.toString(core.dir, si.docCount - numDocs - si.getDelCount());
|
||||
return si.toString(si.dir, si.docCount - numDocs - si.getDelCount());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -196,7 +196,7 @@ public final class SegmentReader extends IndexReader {
|
|||
* Return the name of the segment this reader is reading.
|
||||
*/
|
||||
public String getSegmentName() {
|
||||
return core.segment;
|
||||
return si.name;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -212,7 +212,7 @@ public final class SegmentReader extends IndexReader {
|
|||
// Don't ensureOpen here -- in certain cases, when a
|
||||
// cloned/reopened reader needs to commit, it may call
|
||||
// this method on the closed original reader
|
||||
return core.dir;
|
||||
return si.dir;
|
||||
}
|
||||
|
||||
// This is necessary so that cloned SegmentReaders (which
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
// TODO: we could make separate base class that can wrapp
|
||||
|
@ -137,9 +136,9 @@ public class Lucene40WithOrds extends PostingsFormat {
|
|||
static final String PROX_EXTENSION = "prx";
|
||||
|
||||
@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);
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -202,9 +202,9 @@ public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
SepPostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -225,9 +225,9 @@ public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
SepPostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -413,20 +413,20 @@ public class MockRandomPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SEED_EXT);
|
||||
files.add(seedFileName);
|
||||
SepPostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
BlockTreeTermsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
|
||||
VariableGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
|
||||
// hackish!
|
||||
Iterator<String> it = files.iterator();
|
||||
while(it.hasNext()) {
|
||||
final String file = it.next();
|
||||
if (!dir.fileExists(file)) {
|
||||
if (!segmentInfo.dir.fileExists(file)) {
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.lucene.codecs.sep.SepDocValuesProducer;
|
|||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Separate-file docvalues implementation
|
||||
|
@ -48,7 +47,7 @@ public class MockSepDocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
SepDocValuesConsumer.files(dir, info, files);
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
SepDocValuesConsumer.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.lucene.codecs.sep.SepPostingsWriter;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
|
@ -130,9 +129,9 @@ public class MockSepPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
SepPostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
|
||||
BlockTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
FixedGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.lucene.codecs.pulsing.PulsingPostingsWriter;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Pulsing(1, Pulsing(2, Lucene40))
|
||||
|
@ -92,8 +91,8 @@ public class NestedPulsingPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
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);
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
|
||||
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
|
||||
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,17 +17,23 @@ package org.apache.lucene.codecs.preflexrw;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
|
||||
import org.apache.lucene.codecs.lucene3x.Lucene3xNormsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* Writes 3.x-like indexes (not perfect emulation yet) for testing only!
|
||||
|
@ -35,14 +41,13 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
*/
|
||||
public class PreFlexRWCodec extends Lucene3xCodec {
|
||||
private final PostingsFormat postings = new PreFlexRWPostingsFormat();
|
||||
private final NormsFormat norms = new PreFlexRWNormsFormat();
|
||||
private final Lucene3xNormsFormat norms = new PreFlexRWNormsFormat();
|
||||
private final FieldInfosFormat fieldInfos = new PreFlexRWFieldInfosFormat();
|
||||
private final TermVectorsFormat termVectors = new PreFlexRWTermVectorsFormat();
|
||||
private final SegmentInfosFormat segmentInfos = new PreFlexRWSegmentInfosFormat();
|
||||
private final StoredFieldsFormat storedFields = new PreFlexRWStoredFieldsFormat();
|
||||
// TODO: this should really be a different impl
|
||||
private final LiveDocsFormat liveDocs = new Lucene40LiveDocsFormat();
|
||||
// TODO: this should really be a different impl
|
||||
private final StoredFieldsFormat storedFields = new Lucene40StoredFieldsFormat();
|
||||
|
||||
@Override
|
||||
public PostingsFormat postingsFormat() {
|
||||
|
@ -54,7 +59,7 @@ public class PreFlexRWCodec extends Lucene3xCodec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public NormsFormat normsFormat() {
|
||||
public Lucene3xNormsFormat normsFormat() {
|
||||
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
|
||||
return norms;
|
||||
} else {
|
||||
|
@ -106,4 +111,19 @@ public class PreFlexRWCodec extends Lucene3xCodec {
|
|||
return super.storedFieldsFormat();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getUseCompoundFile() && LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
|
||||
// because we don't fully emulate 3.x codec, PreFlexRW actually writes 4.x format CFS files.
|
||||
// so we must check segment version here to see if its a "real" 3.x segment or a "fake"
|
||||
// one that we wrote with a 4.x-format CFS+CFE, in this case we must add the .CFE
|
||||
String version = info.getVersion();
|
||||
if (version != null && StringHelper.getVersionComparator().compare("4.0", version) <= 0) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
super.files(info, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,17 @@
|
|||
package org.apache.lucene.codecs.preflexrw;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.lucene3x.Lucene3xStoredFieldsFormat;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
public class PreFlexRWStoredFieldsFormat extends Lucene3xStoredFieldsFormat {
|
||||
|
||||
@Override
|
||||
public StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
return new PreFlexRWStoredFieldsWriter(directory, segment, context);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,156 @@
|
|||
package org.apache.lucene.codecs.preflexrw;
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
|
||||
* use this file except in compliance with the License. You may obtain a copy of
|
||||
* the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.lucene3x.Lucene3xStoredFieldsReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public final class PreFlexRWStoredFieldsWriter extends StoredFieldsWriter {
|
||||
private final Directory directory;
|
||||
private final String segment;
|
||||
private IndexOutput fieldsStream;
|
||||
private IndexOutput indexStream;
|
||||
|
||||
public PreFlexRWStoredFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
assert directory != null;
|
||||
this.directory = directory;
|
||||
this.segment = segment;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION), context);
|
||||
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION), context);
|
||||
|
||||
fieldsStream.writeInt(Lucene3xStoredFieldsReader.FORMAT_CURRENT);
|
||||
indexStream.writeInt(Lucene3xStoredFieldsReader.FORMAT_CURRENT);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
abort();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Writes the contents of buffer into the fields stream
|
||||
// and adds a new entry for this document into the index
|
||||
// stream. This assumes the buffer was already written
|
||||
// in the correct fields format.
|
||||
public void startDocument(int numStoredFields) throws IOException {
|
||||
indexStream.writeLong(fieldsStream.getFilePointer());
|
||||
fieldsStream.writeVInt(numStoredFields);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(fieldsStream, indexStream);
|
||||
} finally {
|
||||
fieldsStream = indexStream = null;
|
||||
}
|
||||
}
|
||||
|
||||
public void abort() {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {}
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory,
|
||||
IndexFileNames.segmentFileName(segment, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION),
|
||||
IndexFileNames.segmentFileName(segment, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
public void writeField(FieldInfo info, IndexableField field) throws IOException {
|
||||
fieldsStream.writeVInt(info.number);
|
||||
int bits = 0;
|
||||
final BytesRef bytes;
|
||||
final String string;
|
||||
// TODO: maybe a field should serialize itself?
|
||||
// this way we don't bake into indexer all these
|
||||
// specific encodings for different fields? and apps
|
||||
// can customize...
|
||||
|
||||
Number number = field.numericValue();
|
||||
if (number != null) {
|
||||
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
|
||||
bits |= Lucene3xStoredFieldsReader.FIELD_IS_NUMERIC_INT;
|
||||
} else if (number instanceof Long) {
|
||||
bits |= Lucene3xStoredFieldsReader.FIELD_IS_NUMERIC_LONG;
|
||||
} else if (number instanceof Float) {
|
||||
bits |= Lucene3xStoredFieldsReader.FIELD_IS_NUMERIC_FLOAT;
|
||||
} else if (number instanceof Double) {
|
||||
bits |= Lucene3xStoredFieldsReader.FIELD_IS_NUMERIC_DOUBLE;
|
||||
} else {
|
||||
throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
|
||||
}
|
||||
string = null;
|
||||
bytes = null;
|
||||
} else {
|
||||
bytes = field.binaryValue();
|
||||
if (bytes != null) {
|
||||
bits |= Lucene3xStoredFieldsReader.FIELD_IS_BINARY;
|
||||
string = null;
|
||||
} else {
|
||||
string = field.stringValue();
|
||||
if (string == null) {
|
||||
throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fieldsStream.writeByte((byte) bits);
|
||||
|
||||
if (bytes != null) {
|
||||
fieldsStream.writeVInt(bytes.length);
|
||||
fieldsStream.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
} else if (string != null) {
|
||||
fieldsStream.writeString(field.stringValue());
|
||||
} else {
|
||||
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
|
||||
fieldsStream.writeInt(number.intValue());
|
||||
} else if (number instanceof Long) {
|
||||
fieldsStream.writeLong(number.longValue());
|
||||
} else if (number instanceof Float) {
|
||||
fieldsStream.writeInt(Float.floatToIntBits(number.floatValue()));
|
||||
} else if (number instanceof Double) {
|
||||
fieldsStream.writeLong(Double.doubleToLongBits(number.doubleValue()));
|
||||
} else {
|
||||
assert false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int numDocs) throws IOException {
|
||||
if (4+((long) numDocs)*8 != indexStream.getFilePointer())
|
||||
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
|
||||
// we detect that the bug has struck, here, and
|
||||
// throw an exception to prevent the corruption from
|
||||
// entering the index. See LUCENE-1282 for
|
||||
// details.
|
||||
throw new RuntimeException("fdx size mismatch: docCount is " + numDocs + " but fdx file size is " + indexStream.getFilePointer() + " file=" + indexStream.toString() + "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
}
|
|
@ -202,7 +202,7 @@ public final class PreFlexRWTermVectorsWriter extends TermVectorsWriter {
|
|||
// throw an exception to prevent the corruption from
|
||||
// entering the index. See LUCENE-1282 for
|
||||
// details.
|
||||
throw new RuntimeException("mergeVectors produced an invalid result: mergedDocs is " + numDocs + " but tvx size is " + tvx.getFilePointer() + " file=" + tvx.toString() + "; now aborting this merge to prevent index corruption");
|
||||
throw new RuntimeException("tvx size mismatch: mergedDocs is " + numDocs + " but tvx size is " + tvx.getFilePointer() + " file=" + tvx.toString() + "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
|
||||
/** Close all streams. */
|
||||
|
|
|
@ -46,7 +46,6 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -592,7 +591,7 @@ public class RAMOnlyPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) {
|
||||
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) {
|
||||
final String idFileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, ID_EXTENSION);
|
||||
files.add(idFileName);
|
||||
}
|
||||
|
|
|
@ -1548,7 +1548,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
SegmentInfo s = ((SegmentReader) r).getSegmentInfo();
|
||||
assertFalse(s.getHasVectors());
|
||||
Set<String> files = new HashSet<String>();
|
||||
s.getCodec().termVectorsFormat().files(dir, s, files);
|
||||
s.getCodec().termVectorsFormat().files(s, files);
|
||||
assertTrue(files.isEmpty());
|
||||
}
|
||||
|
||||
|
|
|
@ -1,47 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
public class TestSegmentInfo extends LuceneTestCase {
|
||||
|
||||
public void testSizeInBytesCache() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy());
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("a", "value", TextField.TYPE_STORED));
|
||||
writer.addDocument(doc);
|
||||
writer.close();
|
||||
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(dir);
|
||||
SegmentInfo si = sis.info(0);
|
||||
long sizeInBytesNoStore = si.sizeInBytes(false);
|
||||
long sizeInBytesWithStore = si.sizeInBytes(true);
|
||||
assertTrue("sizeInBytesNoStore=" + sizeInBytesNoStore + " sizeInBytesWithStore=" + sizeInBytesWithStore, sizeInBytesWithStore > sizeInBytesNoStore);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -135,34 +135,4 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
TestSegmentReader.checkNorms(mergedReader);
|
||||
mergedReader.close();
|
||||
}
|
||||
|
||||
// LUCENE-3143
|
||||
public void testInvalidFilesToCreateCompound() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
|
||||
// Create an index w/ .del file
|
||||
w.addDocument(new Document());
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("c", "test"));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
w.deleteDocuments(new Term("c", "test"));
|
||||
w.close();
|
||||
|
||||
// Assert that SM fails if .del exists
|
||||
SegmentMerger sm = new SegmentMerger(InfoStream.getDefault(), dir, 1, "a", MergeState.CheckAbort.NONE, null, null, Codec.getDefault(), newIOContext(random));
|
||||
boolean doFail = false;
|
||||
try {
|
||||
IndexWriter.createCompoundFile(dir, "b1", MergeState.CheckAbort.NONE, w.segmentInfos.info(0), newIOContext(random));
|
||||
doFail = true; // should never get here
|
||||
} catch (AssertionError e) {
|
||||
// expected
|
||||
}
|
||||
assertFalse("should not have been able to create a .cfs with .del and .s* files", doFail);
|
||||
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -189,7 +189,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
|
|||
SegmentInfo s = ((SegmentReader) r).getSegmentInfo();
|
||||
assertTrue(s.getHasVectors());
|
||||
Set<String> files = new HashSet<String>();
|
||||
s.getCodec().termVectorsFormat().files(dir, s, files);
|
||||
s.getCodec().termVectorsFormat().files(s, files);
|
||||
assertFalse(files.isEmpty());
|
||||
for (String file : files) {
|
||||
assertTrue(dir.fileExists(file));
|
||||
|
|
Loading…
Reference in New Issue