mirror of https://github.com/apache/lucene.git
LUCENE-3728: don't pass Directory to files, its confusingly never CFSDir like other Directory parameters, its rarely needed in files(), and redundant with SI.dir
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3661@1237294 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
8d98a6d270
commit
9c22385c55
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ 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.Directory;
|
||||
|
||||
/**
|
||||
* Encodes/decodes an inverted index segment
|
||||
|
@ -47,19 +46,19 @@ 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 {
|
||||
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(dir, info, "", files);
|
||||
storedFieldsFormat().files(dir, info, files);
|
||||
termVectorsFormat().files(dir, info, files);
|
||||
fieldInfosFormat().files(dir, info, files);
|
||||
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(dir, info, files);
|
||||
normsFormat().files(dir, info, files);
|
||||
docValuesFormat().files(info, files);
|
||||
normsFormat().files(info, files);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -67,8 +66,8 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
* stored outside of CFS for the <code>info</code> segment.
|
||||
*/
|
||||
// TODO: can we somehow totally remove this?
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
liveDocsFormat().separateFiles(dir, info, files);
|
||||
public void separateFiles(SegmentInfo info, Set<String> files) throws IOException {
|
||||
liveDocsFormat().separateFiles(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 separateFiles(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,5 +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;
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -88,7 +88,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
|
||||
|
@ -133,20 +133,20 @@ public class Lucene3xCodec extends Codec {
|
|||
|
||||
// overrides the default implementation in codec.java to handle CFS without CFE
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getUseCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
// NOTE: we don't add the CFE extension: because 3.x format doesn't use it.
|
||||
} else {
|
||||
super.files(dir, info, files);
|
||||
super.files(info, files);
|
||||
}
|
||||
}
|
||||
|
||||
// override the default implementation in codec.java to handle separate norms files, and shared compound docstores
|
||||
@Override
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
super.separateFiles(dir, info, files);
|
||||
normsFormat().separateFiles(dir, info, files);
|
||||
public void separateFiles(SegmentInfo info, Set<String> files) throws IOException {
|
||||
super.separateFiles(info, files);
|
||||
normsFormat().separateFiles(info, files);
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
// We are sharing doc stores (stored fields, term
|
||||
// vectors) with other segments
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,7 +117,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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,12 +39,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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -193,11 +193,11 @@ 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);
|
||||
}
|
||||
|
@ -205,7 +205,7 @@ class Lucene3xNormsProducer extends PerDocProducer {
|
|||
|
||||
/** @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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -674,7 +674,7 @@ 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 {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getHasVectors()) {
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
assert info.getDocStoreSegment() != null;
|
||||
|
|
|
@ -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 separateFiles(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));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -286,7 +286,7 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
|
||||
// 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 {
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
assert info.getDocStoreSegment() != null;
|
||||
if (!info.getDocStoreIsCompoundFile()) {
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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 separateFiles(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));
|
||||
}
|
||||
|
|
|
@ -4095,7 +4095,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
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);
|
||||
info.getCodec().separateFiles(info, separateFiles);
|
||||
|
||||
for (String file : files) {
|
||||
assert !separateFiles.contains(file) : file + " should not go in CFS!";
|
||||
|
|
|
@ -258,10 +258,12 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
}
|
||||
|
||||
// nocommit: wrong to call this if (compoundFile)
|
||||
// wrong to call this at all... nuke it
|
||||
Set<String> codecDocStoreFiles() throws IOException {
|
||||
Set<String> docStoreFiles = new HashSet<String>();
|
||||
codec.storedFieldsFormat().files(dir, this, docStoreFiles);
|
||||
codec.termVectorsFormat().files(dir, this, docStoreFiles);
|
||||
codec.storedFieldsFormat().files(this, docStoreFiles);
|
||||
codec.termVectorsFormat().files(this, docStoreFiles);
|
||||
return docStoreFiles;
|
||||
}
|
||||
|
||||
|
@ -472,10 +474,10 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
final Set<String> fileSet = new HashSet<String>();
|
||||
|
||||
codec.files(dir, this, fileSet);
|
||||
codec.files(this, fileSet);
|
||||
|
||||
// regardless of compound file setting: these files are always in the directory
|
||||
codec.separateFiles(dir, this, fileSet);
|
||||
codec.separateFiles(this, fileSet);
|
||||
|
||||
files = new ArrayList<String>(fileSet);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ 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.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
|
@ -115,7 +114,7 @@ public class PreFlexRWCodec extends Lucene3xCodec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
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"
|
||||
|
@ -126,7 +125,7 @@ public class PreFlexRWCodec extends Lucene3xCodec {
|
|||
files.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
}
|
||||
} else {
|
||||
super.files(dir, info, files);
|
||||
super.files(info, files);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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