LUCENE-4055: compute files() at index time not at read time

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1341071 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-05-21 15:32:50 +00:00
parent 437b5c7f5a
commit 19b2a64333
76 changed files with 382 additions and 469 deletions

View File

@ -186,6 +186,7 @@ public class BlockTermsReader extends FieldsProducer {
} }
} }
// nocommit remove?
public static void files(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)); files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockTermsWriter.TERMS_EXTENSION));
} }

View File

@ -199,6 +199,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
} }
} }
// nocommit remove?
public static void files(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_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION)); files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION));

View File

@ -54,28 +54,6 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
return name; return name;
} }
/** Populates <code>files</code> with all filenames needed for
* the <code>info</code> segment.
*/
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);
}
segmentInfosFormat().files(info, files);
// never inside CFS
liveDocsFormat().files(info, files);
}
/** Encodes/decodes postings */ /** Encodes/decodes postings */
public abstract PostingsFormat postingsFormat(); public abstract PostingsFormat postingsFormat();

View File

@ -36,8 +36,4 @@ public abstract class DocValuesFormat {
/** Produces (reads) doc values during reading/searching. */ /** Produces (reads) doc values during reading/searching. */
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException; public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
/** Gathers files (exact file name or a Pattern regex)
* associated with this segment. */
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
} }

View File

@ -35,8 +35,4 @@ public abstract class FieldInfosFormat {
/** Returns a {@link FieldInfosWriter} to write field infos /** Returns a {@link FieldInfosWriter} to write field infos
* to the index */ * to the index */
public abstract FieldInfosWriter getFieldInfosWriter() throws IOException; public abstract FieldInfosWriter getFieldInfosWriter() throws IOException;
/** Gathers files (exact file name or a Pattern regex)
* associated with this segment. */
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
} }

View File

@ -389,6 +389,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
} }
} }
// nocommit remove (and any other files methods!)
public static void files(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)); files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
} }

View File

@ -36,8 +36,4 @@ public abstract class NormsFormat {
/** Returns a {@link PerDocProducer} to read norms from the /** Returns a {@link PerDocProducer} to read norms from the
* index. */ * index. */
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException; public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
/** Gathers files (exact file name or a Pattern regex)
* associated with this segment. */
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
} }

View File

@ -56,15 +56,6 @@ public abstract class PostingsFormat implements NamedSPILoader.NamedSPI {
* use; else, those files may be deleted. */ * use; else, those files may be deleted. */
public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException; public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
/**
* Gathers files (exact file name or a Pattern regex) associated with this segment
*
* @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(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException;
@Override @Override
public String toString() { public String toString() {
return "PostingsFormat(name=" + name + ")"; return "PostingsFormat(name=" + name + ")";
@ -79,5 +70,4 @@ public abstract class PostingsFormat implements NamedSPILoader.NamedSPI {
public static Set<String> availablePostingsFormats() { public static Set<String> availablePostingsFormats() {
return loader.availableServices(); return loader.availableServices();
} }
} }

View File

@ -45,5 +45,4 @@ import org.apache.lucene.index.SegmentInfos; // javadocs
public abstract class SegmentInfosFormat { public abstract class SegmentInfosFormat {
public abstract SegmentInfosReader getSegmentInfosReader(); public abstract SegmentInfosReader getSegmentInfosReader();
public abstract SegmentInfosWriter getSegmentInfosWriter(); public abstract SegmentInfosWriter getSegmentInfosWriter();
public abstract void files(SegmentInfo info, Set<String> files);
} }

View File

@ -36,8 +36,4 @@ public abstract class StoredFieldsFormat {
/** Returns a {@link StoredFieldsWriter} to write stored /** Returns a {@link StoredFieldsWriter} to write stored
* fields. */ * fields. */
public abstract StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException; public abstract StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException;
/** Gathers files (exact file name or a Pattern regex)
* associated with this segment. */
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
} }

View File

@ -36,8 +36,4 @@ public abstract class TermVectorsFormat {
/** Returns a {@link TermVectorsWriter} to write term /** Returns a {@link TermVectorsWriter} to write term
* vectors. */ * vectors. */
public abstract TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException; public abstract TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException;
/** Gathers files (exact file name or a Pattern regex)
* associated with this segment. */
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
} }

View File

@ -80,10 +80,4 @@ class AppendingPostingsFormat extends PostingsFormat {
} }
} }
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
}
} }

View File

@ -84,9 +84,6 @@ public class Lucene3xCodec extends Codec {
public PerDocProducer docsProducer(SegmentReadState state) throws IOException { public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return null; return null;
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {}
}; };
@Override @Override
@ -128,32 +125,4 @@ public class Lucene3xCodec extends Codec {
public LiveDocsFormat liveDocsFormat() { public LiveDocsFormat liveDocsFormat() {
return 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);
((Lucene3xNormsFormat)normsFormat()).separateFiles(info, files);
segmentInfosFormat().files(info, files);
// shared docstores: these guys check the hair
if (info.getDocStoreOffset() != -1) {
storedFieldsFormat().files(info, files);
termVectorsFormat().files(info, files);
}
}
} }

View File

@ -44,9 +44,4 @@ class Lucene3xFieldInfosFormat extends FieldInfosFormat {
public FieldInfosWriter getFieldInfosWriter() throws IOException { public FieldInfosWriter getFieldInfosWriter() throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading"); throw new UnsupportedOperationException("this codec can only be used for reading");
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene3xFieldInfosReader.files(info, files);
}
} }

View File

@ -111,8 +111,4 @@ class Lucene3xFieldInfosReader extends FieldInfosReader {
input.close(); input.close();
} }
} }
public static void files(SegmentInfo info, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(info.name, "", FIELD_INFOS_EXTENSION));
}
} }

View File

@ -36,17 +36,6 @@ import org.apache.lucene.index.SegmentReadState;
@Deprecated @Deprecated
class Lucene3xNormsFormat extends NormsFormat { class Lucene3xNormsFormat extends NormsFormat {
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene3xNormsProducer.files(info, files);
}
public void separateFiles(SegmentInfo info, Set<String> files) throws IOException {
Lucene3xNormsProducer.separateFiles(info, files);
}
@Override @Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading"); throw new UnsupportedOperationException("this codec can only be used for reading");

View File

@ -192,7 +192,8 @@ class Lucene3xNormsProducer extends PerDocProducer {
} }
} }
/*
static void files(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? // 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? // like first FI that has norms but doesn't have separate norms?
@ -202,7 +203,9 @@ class Lucene3xNormsProducer extends PerDocProducer {
files.add(normsFileName); files.add(normsFileName);
} }
} }
*/
/*
static void separateFiles(SegmentInfo info, Set<String> files) throws IOException { static void separateFiles(SegmentInfo info, Set<String> files) throws IOException {
Map<Integer,Long> normGen = info.getNormGen(); Map<Integer,Long> normGen = info.getNormGen();
if (normGen != null) { if (normGen != null) {
@ -215,6 +218,7 @@ class Lucene3xNormsProducer extends PerDocProducer {
} }
} }
} }
*/
private class NormsDocValues extends DocValues { private class NormsDocValues extends DocValues {
private final IndexInput file; private final IndexInput file;

View File

@ -63,10 +63,4 @@ class Lucene3xPostingsFormat extends PostingsFormat {
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor); return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
} }
@Override
public void files(SegmentInfo info, String segmentSuffix, Set<String> files) throws IOException {
// preflex fields have no segmentSuffix - we ignore it here
Lucene3xFields.files(info, files);
}
} }

View File

@ -62,11 +62,4 @@ public class Lucene3xSegmentInfosFormat extends SegmentInfosFormat {
public SegmentInfosWriter getSegmentInfosWriter() { public SegmentInfosWriter getSegmentInfosWriter() {
throw new UnsupportedOperationException("this codec can only be used for reading"); throw new UnsupportedOperationException("this codec can only be used for reading");
} }
@Override
public void files(SegmentInfo info, Set<String> files) {
// nocommit must take care to filter this out if we are
// "really" an old 3.x index
files.add(IndexFileNames.segmentFileName(info.name, "", SI_EXTENSION));
}
} }

View File

@ -18,8 +18,11 @@ package org.apache.lucene.codecs.lucene3x;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; // nocommit
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.SegmentInfosReader; import org.apache.lucene.codecs.SegmentInfosReader;
@ -115,6 +118,12 @@ public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
} }
} }
private static void addIfExists(Directory dir, Set<String> files, String fileName) throws IOException {
if (dir.fileExists(fileName)) {
files.add(fileName);
}
}
private SegmentInfo readSegmentInfo(String segmentName, Directory dir, int format, IndexInput input) throws IOException { private SegmentInfo readSegmentInfo(String segmentName, Directory dir, int format, IndexInput input) throws IOException {
// check that it is a format we can understand // check that it is a format we can understand
if (format > Lucene3xSegmentInfosFormat.FORMAT_DIAGNOSTICS) { if (format > Lucene3xSegmentInfosFormat.FORMAT_DIAGNOSTICS) {
@ -177,8 +186,68 @@ public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
final Map<String,String> diagnostics = input.readStringStringMap(); final Map<String,String> diagnostics = input.readStringStringMap();
// nocommit unused...
final int hasVectors;
if (format <= Lucene3xSegmentInfosFormat.FORMAT_HAS_VECTORS) { if (format <= Lucene3xSegmentInfosFormat.FORMAT_HAS_VECTORS) {
input.readByte(); hasVectors = input.readByte();
} else {
hasVectors = -1;
}
final Set<String> files;
if (format == Lucene3xSegmentInfosFormat.FORMAT_4X_UPGRADE) {
files = input.readStringSet();
} else {
// Replicate logic from 3.x's SegmentInfo.files():
files = new HashSet<String>();
if (isCompoundFile) {
files.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
} else {
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "fnm"));
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "frq"));
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "prx"));
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "tis"));
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "tii"));
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "nrm"));
}
if (docStoreOffset != -1) {
if (docStoreIsCompoundFile) {
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", "cfx"));
} else {
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", "fdx"));
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", "fdt"));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", "tvx"));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", "tvf"));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", "tvd"));
}
} else if (!isCompoundFile) {
files.add(IndexFileNames.segmentFileName(segmentName, "", "fdx"));
files.add(IndexFileNames.segmentFileName(segmentName, "", "fdt"));
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "tvx"));
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "tvf"));
addIfExists(dir, files, IndexFileNames.segmentFileName(segmentName, "", "tvd"));
}
if (normGen != null) {
for(Map.Entry<Integer,Long> ent : normGen.entrySet()) {
long gen = ent.getValue();
if (gen >= SegmentInfo.YES) {
// Definitely a separate norm file, with generation:
files.add(IndexFileNames.fileNameFromGeneration(segmentName, "s" + ent.getKey(), gen));
} else if (gen == SegmentInfo.NO) {
// No seaprate norm
} else {
// nocommit -- i thought _X_N.sY files were pre-3.0...????
assert false;
/*
System.out.println("FILES: " + Arrays.toString(dir.listAll()) + "; seg=" + segmentName);
addIfExists(dir, files, IndexFileNames.fileNameFromGeneration(segmentName, "s" + ent.getKey(), gen));
assert false: "gen=" + gen;
*/
}
}
}
} }
// nocommit we can use hasProx/hasVectors from the 3.x // nocommit we can use hasProx/hasVectors from the 3.x
@ -188,6 +257,7 @@ public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile, docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
delCount, null, diagnostics); delCount, null, diagnostics);
info.setDelGen(delGen); info.setDelGen(delGen);
info.setFiles(files);
return info; return info;
} }
} }

View File

@ -43,9 +43,4 @@ class Lucene3xStoredFieldsFormat extends StoredFieldsFormat {
IOContext context) throws IOException { IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading"); 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);
}
} }

View File

@ -299,6 +299,7 @@ final class Lucene3xStoredFieldsReader extends StoredFieldsReader implements Clo
// note: if there are shared docstores, we are also called by Lucene3xCodec even in // note: if there are shared docstores, we are also called by Lucene3xCodec even in
// the CFS case. so logic here must handle this. // the CFS case. so logic here must handle this.
/*
public static void files(SegmentInfo info, Set<String> files) throws IOException { public static void files(SegmentInfo info, Set<String> files) throws IOException {
if (info.getDocStoreOffset() != -1) { if (info.getDocStoreOffset() != -1) {
assert info.getDocStoreSegment() != null; assert info.getDocStoreSegment() != null;
@ -313,4 +314,5 @@ final class Lucene3xStoredFieldsReader extends StoredFieldsReader implements Clo
files.add(IndexFileNames.segmentFileName(info.name, "", FIELDS_EXTENSION)); files.add(IndexFileNames.segmentFileName(info.name, "", FIELDS_EXTENSION));
} }
} }
*/
} }

View File

@ -78,10 +78,4 @@ class Lucene3xTermVectorsFormat extends TermVectorsFormat {
public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException { public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading"); throw new UnsupportedOperationException("this codec can only be used for reading");
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene3xTermVectorsReader.files(info, files);
}
} }

View File

@ -692,6 +692,7 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
// note: if there are shared docstores, we are also called by Lucene3xCodec even in // note: if there are shared docstores, we are also called by Lucene3xCodec even in
// the CFS case. so logic here must handle this. // the CFS case. so logic here must handle this.
/*
public static void files(SegmentInfo info, Set<String> files) throws IOException { public static void files(SegmentInfo info, Set<String> files) throws IOException {
if (info.getDocStoreOffset() != -1) { if (info.getDocStoreOffset() != -1) {
assert info.getDocStoreSegment() != null; assert info.getDocStoreSegment() != null;
@ -708,6 +709,7 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION)); files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
} }
} }
*/
// If this returns, we do the surrogates shuffle so that the // If this returns, we do the surrogates shuffle so that the
// terms are sorted by unicode sort order. This should be // terms are sorted by unicode sort order. This should be

View File

@ -66,11 +66,6 @@ public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
} }
} }
public static void files(SegmentInfo segmentInfo, Set<String> files) throws IOException {
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));
}
@Override @Override
public void abort() { public void abort() {
try { try {

View File

@ -141,9 +141,4 @@ public class Lucene40DocValuesFormat extends DocValuesFormat {
public PerDocProducer docsProducer(SegmentReadState state) throws IOException { public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new Lucene40DocValuesProducer(state, Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX); return new Lucene40DocValuesProducer(state, Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX);
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene40DocValuesConsumer.files(info, files);
}
} }

View File

@ -107,9 +107,4 @@ public class Lucene40FieldInfosFormat extends FieldInfosFormat {
public FieldInfosWriter getFieldInfosWriter() throws IOException { public FieldInfosWriter getFieldInfosWriter() throws IOException {
return writer; return writer;
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene40FieldInfosReader.files(info, files);
}
} }

View File

@ -140,8 +140,4 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
throw new IllegalStateException("unhandled indexValues type " + b); throw new IllegalStateException("unhandled indexValues type " + b);
} }
} }
public static void files(SegmentInfo info, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(info.name, "", Lucene40FieldInfosWriter.FIELD_INFOS_EXTENSION));
}
} }

View File

@ -59,11 +59,6 @@ public class Lucene40NormsFormat extends NormsFormat {
return new Lucene40NormsDocValuesProducer(state, NORMS_SEGMENT_SUFFIX); return new Lucene40NormsDocValuesProducer(state, NORMS_SEGMENT_SUFFIX);
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene40NormsDocValuesConsumer.files(info, files);
}
/** /**
* Lucene 4.0 PerDocProducer implementation that uses compound file. * Lucene 4.0 PerDocProducer implementation that uses compound file.
* *

View File

@ -334,12 +334,6 @@ public class Lucene40PostingsFormat extends PostingsFormat {
/** Extension of prox postings file */ /** Extension of prox postings file */
static final String PROX_EXTENSION = "prx"; static final String PROX_EXTENSION = "prx";
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
}
@Override @Override
public String toString() { public String toString() {
return getName() + "(minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")"; return getName() + "(minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";

View File

@ -136,9 +136,4 @@ public class Lucene40SegmentInfosFormat extends SegmentInfosFormat {
} }
public final static String SI_EXTENSION = "si"; public final static String SI_EXTENSION = "si";
@Override
public void files(SegmentInfo segmentInfo, Set<String> files) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", SI_EXTENSION));
}
} }

View File

@ -19,7 +19,9 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.SegmentInfosReader; import org.apache.lucene.codecs.SegmentInfosReader;
@ -54,10 +56,13 @@ public class Lucene40SegmentInfosReader extends SegmentInfosReader {
final Map<Integer,Long> normGen = null; final Map<Integer,Long> normGen = null;
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES; final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final Map<String,String> diagnostics = input.readStringStringMap(); final Map<String,String> diagnostics = input.readStringStringMap();
final Set<String> files = input.readStringSet();
final SegmentInfo si = new SegmentInfo(dir, version, segment, docCount, docStoreOffset, final SegmentInfo si = new SegmentInfo(dir, version, segment, docCount, docStoreOffset,
docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile, docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
0, null, diagnostics); 0, null, diagnostics);
si.setFiles(files);
success = true; success = true;
return si; return si;

View File

@ -18,8 +18,10 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.HashSet;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.SegmentInfosWriter; import org.apache.lucene.codecs.SegmentInfosWriter;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
@ -46,6 +48,9 @@ public class Lucene40SegmentInfosWriter extends SegmentInfosWriter {
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException { public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
assert si.getDelCount() <= si.docCount: "delCount=" + si.getDelCount() + " docCount=" + si.docCount + " segment=" + si.name; assert si.getDelCount() <= si.docCount: "delCount=" + si.getDelCount() + " docCount=" + si.docCount + " segment=" + si.name;
final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene40SegmentInfosFormat.SI_EXTENSION); final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene40SegmentInfosFormat.SI_EXTENSION);
assert si.getFiles() != null;
si.getFiles().add(fileName);
final IndexOutput output = dir.createOutput(fileName, ioContext); final IndexOutput output = dir.createOutput(fileName, ioContext);
boolean success = false; boolean success = false;
@ -59,6 +64,7 @@ public class Lucene40SegmentInfosWriter extends SegmentInfosWriter {
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
output.writeStringStringMap(si.getDiagnostics()); output.writeStringStringMap(si.getDiagnostics());
output.writeStringSet(si.getFiles());
success = true; success = true;
} finally { } finally {

View File

@ -90,9 +90,4 @@ public class Lucene40StoredFieldsFormat extends StoredFieldsFormat {
IOContext context) throws IOException { IOContext context) throws IOException {
return new Lucene40StoredFieldsWriter(directory, segment, context); return new Lucene40StoredFieldsWriter(directory, segment, context);
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene40StoredFieldsReader.files(info, files);
}
} }

View File

@ -240,9 +240,4 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
return fieldsStream; return fieldsStream;
} }
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));
}
} }

View File

@ -111,9 +111,4 @@ public class Lucene40TermVectorsFormat extends TermVectorsFormat {
public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException { public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
return new Lucene40TermVectorsWriter(directory, segment, context); return new Lucene40TermVectorsWriter(directory, segment, context);
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene40TermVectorsReader.files(info, files);
}
} }

View File

@ -710,11 +710,5 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, format); return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, format);
} }
public static void files(SegmentInfo info, Set<String> files) throws IOException {
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));
}
} }

View File

@ -901,9 +901,4 @@ public class MemoryPostingsFormat extends PostingsFormat {
} }
}; };
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION));
}
} }

View File

@ -332,34 +332,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
protected abstract void visitOneField(String fieldName, PostingsFormat format) throws IOException; protected abstract void visitOneField(String fieldName, PostingsFormat format) throws IOException;
} }
@Override // nocommit simplify now that we don't have files()...?
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);
// nocommit can we use regexp to simplify this?
try {
new VisitPerFieldFile(dir, info.name, segmentSuffix) {
@Override
protected void visitOneFormat(String segmentSuffix, PostingsFormat format) throws IOException {
format.files(info, segmentSuffix, files);
}
@Override
protected void visitOneField(String field, PostingsFormat format) {
}
};
} catch (FileNotFoundException fnfe) {
// TODO: this is somewhat shady... if we can't open
// the .per file then most likely someone is calling
// .files() after this segment was deleted, so, they
// wouldn't be able to do anything with the files even
// if we could return them, so we don't add any files
// in this case.
}
}
// NOTE: only called during writing; for reading we read // NOTE: only called during writing; for reading we read
// all we need from the index (ie we save the field -> // all we need from the index (ie we save the field ->

View File

@ -112,10 +112,4 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
public int getFreqCutoff() { public int getFreqCutoff() {
return freqCutoff; return freqCutoff;
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
wrappedPostingsBaseFormat.files(segmentInfo, segmentSuffix, files);
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
}
} }

View File

@ -49,11 +49,6 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
return directory; return directory;
} }
public static void files(SegmentInfo segmentInfo,
Set<String> files) throws IOException {
files(segmentInfo, files);
}
@SuppressWarnings("fallthrough") @SuppressWarnings("fallthrough")
private static void files(String segmentName, Set<String> files) { private static void files(String segmentName, Set<String> files) {
String filename = PerDocProducerBase.docValuesRegex(segmentName); String filename = PerDocProducerBase.docValuesRegex(segmentName);

View File

@ -49,10 +49,4 @@ public class SimpleTextDocValuesFormat extends DocValuesFormat {
static String docValuesId(String segmentsName, int fieldId) { static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId; return segmentsName + "_" + fieldId;
} }
@Override
public void files(SegmentInfo info, Set<String> files)
throws IOException {
SimpleTextPerDocConsumer.files(info, files, DOC_VALUES_SEG_SUFFIX);
}
} }

View File

@ -44,9 +44,4 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
public FieldInfosWriter getFieldInfosWriter() throws IOException { public FieldInfosWriter getFieldInfosWriter() throws IOException {
return writer; return writer;
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
SimpleTextFieldInfosReader.files(info, files);
}
} }

View File

@ -124,8 +124,4 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
private String readString(int offset, BytesRef scratch) { private String readString(int offset, BytesRef scratch) {
return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8); return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
} }
public static void files(SegmentInfo info, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(info.name, "", FIELD_INFOS_EXTENSION));
}
} }

View File

@ -58,11 +58,6 @@ public class SimpleTextNormsFormat extends NormsFormat {
BytesRef.getUTF8SortedAsUnicodeComparator()); BytesRef.getUTF8SortedAsUnicodeComparator());
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
SimpleTextNormsPerDocConsumer.files(info, files);
}
/** /**
* Reads plain-text norms. * Reads plain-text norms.
* <p> * <p>
@ -134,11 +129,6 @@ public class SimpleTextNormsFormat extends NormsFormat {
SegmentInfo.findMatchingFiles(state.segmentName, state.directory, files).toArray(new String[0])); SegmentInfo.findMatchingFiles(state.segmentName, state.directory, files).toArray(new String[0]));
} }
public static void files(SegmentInfo segmentInfo, Set<String> files)
throws IOException {
filesInternal(segmentInfo.name, files);
}
public static void filesInternal(String segmentName, public static void filesInternal(String segmentName,
Set<String> files) { Set<String> files) {
String id = docValuesIdRegexp(segmentName); String id = docValuesIdRegexp(segmentName);

View File

@ -63,10 +63,6 @@ class SimpleTextPerDocConsumer extends PerDocConsumer {
SegmentInfo.findMatchingFiles(state.segmentName, state.directory, files).toArray(new String[0])); SegmentInfo.findMatchingFiles(state.segmentName, state.directory, files).toArray(new String[0]));
} }
static void files(SegmentInfo info, Set<String> files, String segmentSuffix) {
files(info.dir, info.name, files, segmentSuffix);
}
static String docValuesId(String segmentsName, int fieldId) { static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId; return segmentsName + "_" + fieldId;
} }

View File

@ -58,9 +58,4 @@ public class SimpleTextPostingsFormat extends PostingsFormat {
static String getPostingsFileName(String segment, String segmentSuffix) { static String getPostingsFileName(String segment, String segmentSuffix) {
return IndexFileNames.segmentFileName(segment, segmentSuffix, POSTINGS_EXTENSION); return IndexFileNames.segmentFileName(segment, segmentSuffix, POSTINGS_EXTENSION);
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
files.add(getPostingsFileName(segmentInfo.name, segmentSuffix));
}
} }

View File

@ -46,9 +46,4 @@ public class SimpleTextSegmentInfosFormat extends SegmentInfosFormat {
public SegmentInfosWriter getSegmentInfosWriter() { public SegmentInfosWriter getSegmentInfosWriter() {
return writer; return writer;
} }
@Override
public void files(SegmentInfo info, Set<String> files) {
files.add(IndexFileNames.segmentFileName(info.name, "", SI_EXTENSION));
}
} }

View File

@ -19,7 +19,9 @@ package org.apache.lucene.codecs.simpletext;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.SegmentInfosReader; import org.apache.lucene.codecs.SegmentInfosReader;
@ -49,8 +51,8 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader {
@Override @Override
public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException { public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
BytesRef scratch = new BytesRef(); BytesRef scratch = new BytesRef();
String fileName = IndexFileNames.segmentFileName(segmentName, "", SimpleTextSegmentInfosFormat.SI_EXTENSION); String segFileName = IndexFileNames.segmentFileName(segmentName, "", SimpleTextSegmentInfosFormat.SI_EXTENSION);
IndexInput input = directory.openInput(fileName, context); IndexInput input = directory.openInput(segFileName, context);
boolean success = false; boolean success = false;
try { try {
SimpleTextUtil.readLine(input, scratch); SimpleTextUtil.readLine(input, scratch);
@ -81,10 +83,24 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader {
diagnostics.put(key, value); diagnostics.put(key, value);
} }
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_NUM_FILES);
int numFiles = Integer.parseInt(readString(SI_NUM_FILES.length, scratch));
Set<String> files = new HashSet<String>();
for (int i = 0; i < numFiles; i++) {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_FILE);
String fileName = readString(SI_FILE.length, scratch);
files.add(fileName);
}
SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount, -1,
segmentName, false, null, isCompoundFile,
0, null, diagnostics);
info.setFiles(files);
success = true; success = true;
return new SegmentInfo(directory, version, segmentName, docCount, -1, return info;
segmentName, false, null, isCompoundFile,
0, null, diagnostics);
} finally { } finally {
if (!success) { if (!success) {
IOUtils.closeWhileHandlingException(input); IOUtils.closeWhileHandlingException(input);

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.simpletext;
import java.io.IOException; import java.io.IOException;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.SegmentInfosWriter; import org.apache.lucene.codecs.SegmentInfosWriter;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
@ -48,14 +49,18 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter {
final static BytesRef SI_NUM_DIAG = new BytesRef(" diagnostics "); final static BytesRef SI_NUM_DIAG = new BytesRef(" diagnostics ");
final static BytesRef SI_DIAG_KEY = new BytesRef(" key "); final static BytesRef SI_DIAG_KEY = new BytesRef(" key ");
final static BytesRef SI_DIAG_VALUE = new BytesRef(" value "); final static BytesRef SI_DIAG_VALUE = new BytesRef(" value ");
final static BytesRef SI_NUM_FILES = new BytesRef(" files ");
final static BytesRef SI_FILE = new BytesRef(" file ");
@Override @Override
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException { public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
assert si.getDelCount() <= si.docCount: "delCount=" + si.getDelCount() + " docCount=" + si.docCount + " segment=" + si.name; assert si.getDelCount() <= si.docCount: "delCount=" + si.getDelCount() + " docCount=" + si.docCount + " segment=" + si.name;
String fileName = IndexFileNames.segmentFileName(si.name, "", SimpleTextSegmentInfosFormat.SI_EXTENSION); String segFileName = IndexFileNames.segmentFileName(si.name, "", SimpleTextSegmentInfosFormat.SI_EXTENSION);
si.getFiles().add(segFileName);
boolean success = false; boolean success = false;
IndexOutput output = dir.createOutput(fileName, ioContext); IndexOutput output = dir.createOutput(segFileName, ioContext);
try { try {
BytesRef scratch = new BytesRef(); BytesRef scratch = new BytesRef();
@ -89,6 +94,20 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter {
SimpleTextUtil.writeNewline(output); SimpleTextUtil.writeNewline(output);
} }
} }
Set<String> files = si.getFiles();
int numFiles = files == null ? 0 : files.size();
SimpleTextUtil.write(output, SI_NUM_FILES);
SimpleTextUtil.write(output, Integer.toString(numFiles), scratch);
SimpleTextUtil.writeNewline(output);
if (numFiles > 0) {
for(String fileName : files) {
SimpleTextUtil.write(output, SI_FILE);
SimpleTextUtil.write(output, fileName, scratch);
SimpleTextUtil.writeNewline(output);
}
}
success = true; success = true;
} finally { } finally {
if (!success) { if (!success) {

View File

@ -45,9 +45,4 @@ public class SimpleTextStoredFieldsFormat extends StoredFieldsFormat {
public StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException { public StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
return new SimpleTextStoredFieldsWriter(directory, segment, context); return new SimpleTextStoredFieldsWriter(directory, segment, context);
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
SimpleTextStoredFieldsReader.files(info, files);
}
} }

View File

@ -178,10 +178,6 @@ public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
} }
} }
public static void files(SegmentInfo info, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(info.name, "", SimpleTextStoredFieldsWriter.FIELDS_EXTENSION));
}
private void readLine() throws IOException { private void readLine() throws IOException {
SimpleTextUtil.readLine(in, scratch); SimpleTextUtil.readLine(in, scratch);
} }

View File

@ -45,9 +45,4 @@ public class SimpleTextTermVectorsFormat extends TermVectorsFormat {
public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException { public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
return new SimpleTextTermVectorsWriter(directory, segment, context); return new SimpleTextTermVectorsWriter(directory, segment, context);
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
SimpleTextTermVectorsReader.files(info, files);
}
} }

View File

@ -200,11 +200,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
offsets = null; offsets = null;
} }
} }
public static void files(SegmentInfo info, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_EXTENSION));
}
private void readLine() throws IOException { private void readLine() throws IOException {
SimpleTextUtil.readLine(in, scratch); SimpleTextUtil.readLine(in, scratch);
} }

View File

@ -19,17 +19,20 @@ package org.apache.lucene.index;
import java.io.IOException; import java.io.IOException;
import java.text.NumberFormat; import java.text.NumberFormat;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.util.Constants;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice; import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo; import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.ByteBlockPool.Allocator; import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator; import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.Counter; import org.apache.lucene.util.Counter;
import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.MutableBits; import org.apache.lucene.util.MutableBits;
@ -160,7 +163,8 @@ class DocumentsWriterPerThread {
final DocumentsWriter parent; final DocumentsWriter parent;
final Codec codec; final Codec codec;
final IndexWriter writer; final IndexWriter writer;
final Directory directory; final TrackingDirectoryWrapper directory;
final Directory directoryOrig;
final DocState docState; final DocState docState;
final DocConsumer consumer; final DocConsumer consumer;
final Counter bytesUsed; final Counter bytesUsed;
@ -184,7 +188,8 @@ class DocumentsWriterPerThread {
public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent, public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent,
FieldInfos.Builder fieldInfos, IndexingChain indexingChain) { FieldInfos.Builder fieldInfos, IndexingChain indexingChain) {
this.directory = directory; this.directoryOrig = directory;
this.directory = new TrackingDirectoryWrapper(directory);
this.parent = parent; this.parent = parent;
this.fieldInfos = fieldInfos; this.fieldInfos = fieldInfos;
this.writer = parent.indexWriter; this.writer = parent.indexWriter;
@ -200,7 +205,7 @@ class DocumentsWriterPerThread {
} }
public DocumentsWriterPerThread(DocumentsWriterPerThread other, FieldInfos.Builder fieldInfos) { public DocumentsWriterPerThread(DocumentsWriterPerThread other, FieldInfos.Builder fieldInfos) {
this(other.directory, other.parent, fieldInfos, other.parent.chain); this(other.directoryOrig, other.parent, fieldInfos, other.parent.chain);
} }
void initialize() { void initialize() {
@ -416,6 +421,7 @@ class DocumentsWriterPerThread {
private void doAfterFlush() throws IOException { private void doAfterFlush() throws IOException {
segment = null; segment = null;
consumer.doAfterFlush(); consumer.doAfterFlush();
directory.getCreatedFiles().clear();
fieldInfos = new FieldInfos.Builder(fieldInfos.globalFieldNumbers); fieldInfos = new FieldInfos.Builder(fieldInfos.globalFieldNumbers);
parent.subtractFlushedNumDocs(numDocsInRAM); parent.subtractFlushedNumDocs(numDocsInRAM);
numDocsInRAM = 0; numDocsInRAM = 0;
@ -448,6 +454,7 @@ class DocumentsWriterPerThread {
numDocsInRAM, writer.getConfig().getTermIndexInterval(), numDocsInRAM, writer.getConfig().getTermIndexInterval(),
codec, pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed()))); codec, pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.; final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
// Apply delete-by-docID now (delete-byDocID only // Apply delete-by-docID now (delete-byDocID only
// happens when an exception is hit processing that // happens when an exception is hit processing that
// doc, eg if analyzer has some problem w/ the text): // doc, eg if analyzer has some problem w/ the text):
@ -477,10 +484,11 @@ class DocumentsWriterPerThread {
try { try {
consumer.flush(flushState); consumer.flush(flushState);
pendingDeletes.terms.clear(); pendingDeletes.terms.clear();
final SegmentInfo newSegment = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, segment, flushState.numDocs, final SegmentInfo newSegment = new SegmentInfo(directoryOrig, Constants.LUCENE_MAIN_VERSION, segment, flushState.numDocs,
-1, segment, false, null, false, 0, -1, segment, false, null, false, 0,
flushState.codec, flushState.codec,
null); null);
newSegment.setFiles(new HashSet<String>(directory.getCreatedFiles()));
if (infoStream.isEnabled("DWPT")) { if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.delCountOnFlush)) + " deleted docs"); infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.delCountOnFlush)) + " deleted docs");

View File

@ -103,7 +103,8 @@ final class IndexFileDeleter {
/** Change to true to see details of reference counts when /** Change to true to see details of reference counts when
* infoStream is enabled */ * infoStream is enabled */
public static boolean VERBOSE_REF_COUNTS = false; // nocommit back to false:
public static boolean VERBOSE_REF_COUNTS = true;
// Used only for assert // Used only for assert
private final IndexWriter writer; private final IndexWriter writer;

View File

@ -504,7 +504,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
*/ */
public synchronized ReadersAndLiveDocs get(SegmentInfo info, boolean create) { public synchronized ReadersAndLiveDocs get(SegmentInfo info, boolean create) {
assert info.dir == directory; assert info.dir == directory: "info.dir=" + info.dir + " vs " + directory;
ReadersAndLiveDocs rld = readerMap.get(info); ReadersAndLiveDocs rld = readerMap.get(info);
if (rld == null) { if (rld == null) {
@ -2024,14 +2024,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
boolean success = false; boolean success = false;
try { try {
if (useCompoundFile(newSegment)) { if (useCompoundFile(newSegment)) {
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
// Now build compound file // Now build compound file
Collection<String> files = createCompoundFile(infoStream, directory, compoundFileName, MergeState.CheckAbort.NONE, newSegment, context); Collection<String> oldFiles = createCompoundFile(infoStream, directory, MergeState.CheckAbort.NONE, newSegment, context);
newSegment.setUseCompoundFile(true); newSegment.setUseCompoundFile(true);
synchronized(this) { synchronized(this) {
deleter.deleteNewFiles(files); deleter.deleteNewFiles(oldFiles);
} }
} }
@ -2278,12 +2277,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
String mergedName = newSegmentName(); String mergedName = newSegmentName();
for (IndexReader indexReader : readers) { for (IndexReader indexReader : readers) {
numDocs += indexReader.numDocs(); numDocs += indexReader.numDocs();
} }
final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, -1)); final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, -1));
// TODO: somehow we should fix this merge so it's // TODO: somehow we should fix this merge so it's
// abortable so that IW.close(false) is able to stop it // abortable so that IW.close(false) is able to stop it
SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
SegmentMerger merger = new SegmentMerger(infoStream, trackingDir, config.getTermIndexInterval(),
mergedName, MergeState.CheckAbort.NONE, payloadProcessorProvider, mergedName, MergeState.CheckAbort.NONE, payloadProcessorProvider,
new FieldInfos.Builder(globalFieldNumberMap), codec, context); new FieldInfos.Builder(globalFieldNumberMap), codec, context);
@ -2296,6 +2296,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, docCount, SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, docCount,
-1, mergedName, false, null, false, 0, -1, mergedName, false, null, false, 0,
codec, null); codec, null);
info.setFiles(new HashSet<String>(trackingDir.getCreatedFiles()));
trackingDir.getCreatedFiles().clear();
setDiagnostics(info, "addIndexes(IndexReader...)"); setDiagnostics(info, "addIndexes(IndexReader...)");
@ -2311,12 +2313,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Now create the compound file if needed // Now create the compound file if needed
if (useCompoundFile) { if (useCompoundFile) {
createCompoundFile(infoStream, directory, IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION), MergeState.CheckAbort.NONE, info, context); Collection<String> filesToDelete = info.files();
createCompoundFile(infoStream, directory, MergeState.CheckAbort.NONE, info, context);
// delete new non cfs files directly: they were never // delete new non cfs files directly: they were never
// registered with IFD // registered with IFD
synchronized(this) { synchronized(this) {
deleter.deleteNewFiles(info.files()); deleter.deleteNewFiles(filesToDelete);
} }
info.setUseCompoundFile(true); info.setUseCompoundFile(true);
} }
@ -2325,8 +2328,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// creating CFS so that 1) .si isn't slurped into CFS, // creating CFS so that 1) .si isn't slurped into CFS,
// and 2) .si reflects useCompoundFile=true change // and 2) .si reflects useCompoundFile=true change
// above: // above:
codec.segmentInfosFormat().getSegmentInfosWriter().write(directory, info, mergeState.fieldInfos, context); codec.segmentInfosFormat().getSegmentInfosWriter().write(trackingDir, info, mergeState.fieldInfos, context);
info.clearFilesCache(); info.clearFilesCache();
info.getFiles().addAll(trackingDir.getCreatedFiles());
// Register the new segment // Register the new segment
synchronized(this) { synchronized(this) {
@ -2369,10 +2373,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
Lucene3xSegmentInfosFormat.SI_EXTENSION); Lucene3xSegmentInfosFormat.SI_EXTENSION);
if (hasSharedDocStore) { if (hasSharedDocStore) {
// only violate the codec this way if its preflex & // only violate the codec this way if it's preflex &
// shares doc stores // shares doc stores
info.getCodec().storedFieldsFormat().files(info, codecDocStoreFiles); assert info.getDocStoreSegment() != null;
info.getCodec().termVectorsFormat().files(info, codecDocStoreFiles); // nocommit what to do....
if (info.getDocStoreIsCompoundFile()) {
codecDocStoreFiles.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", "cfx"));
} else {
codecDocStoreFiles.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", "fdt"));
codecDocStoreFiles.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", "fdx"));
codecDocStoreFiles.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", "tvx"));
codecDocStoreFiles.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", "tvf"));
codecDocStoreFiles.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", "tvd"));
}
} }
//System.out.println("copy seg=" + info.name + " version=" + info.getVersion()); //System.out.println("copy seg=" + info.name + " version=" + info.getVersion());
@ -2383,18 +2396,30 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
info.getDelCount(), info.getCodec(), info.getDiagnostics()); info.getDelCount(), info.getCodec(), info.getDiagnostics());
newInfo.setDelGen(info.getDelGen()); newInfo.setDelGen(info.getDelGen());
Set<String> segFiles = new HashSet<String>();
// Build up new segment's file names:
for (String file: info.files()) {
final String newFileName;
if (codecDocStoreFiles.contains(file)) {
newFileName = newDsName + IndexFileNames.stripSegmentName(file);
} else {
newFileName = segName + IndexFileNames.stripSegmentName(file);
}
segFiles.add(newFileName);
}
newInfo.setFiles(segFiles);
// We must rewrite the SI file because it references // We must rewrite the SI file because it references
// segment name (its own name, if its 3.x, and doc // segment name (its own name, if its 3.x, and doc
// store segment name): // store segment name):
TrackingDirectoryWrapper dirWrapper = new TrackingDirectoryWrapper(directory);
try { try {
newInfo.getCodec().segmentInfosFormat().getSegmentInfosWriter().write(dirWrapper, newInfo, null, context); newInfo.getCodec().segmentInfosFormat().getSegmentInfosWriter().write(directory, newInfo, null, context);
} catch (UnsupportedOperationException uoe) { } catch (UnsupportedOperationException uoe) {
// OK: 3x codec cannot write a new SI file; // OK: 3x codec cannot write a new SI file;
// SegmentInfos will write this on commit // SegmentInfos will write this on commit
} }
final Set<String> siFileNames = dirWrapper.getCreatedFiles();
// Copy the segment's files // Copy the segment's files
for (String file: info.files()) { for (String file: info.files()) {
@ -2409,8 +2434,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
newFileName = segName + IndexFileNames.stripSegmentName(file); newFileName = segName + IndexFileNames.stripSegmentName(file);
} }
if (siFileNames != null && siFileNames.contains(newFileName)) { // nocommit hack
// We already rewwrote this above //if (siFileNames != null && siFileNames.contains(newFileName)) {
if (newFileName.endsWith(".si")) {
// We already rewrote this above
continue; continue;
} }
@ -3412,7 +3439,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
IOContext context = new IOContext(merge.getMergeInfo()); IOContext context = new IOContext(merge.getMergeInfo());
final MergeState.CheckAbort checkAbort = new MergeState.CheckAbort(merge, directory); final MergeState.CheckAbort checkAbort = new MergeState.CheckAbort(merge, directory);
SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), mergedName, checkAbort, final TrackingDirectoryWrapper dirWrapper = new TrackingDirectoryWrapper(directory);
SegmentMerger merger = new SegmentMerger(infoStream, dirWrapper, config.getTermIndexInterval(), mergedName, checkAbort,
payloadProcessorProvider, new FieldInfos.Builder(globalFieldNumberMap), codec, context); payloadProcessorProvider, new FieldInfos.Builder(globalFieldNumberMap), codec, context);
if (infoStream.isEnabled("IW")) { if (infoStream.isEnabled("IW")) {
@ -3474,6 +3502,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// This is where all the work happens: // This is where all the work happens:
MergeState mergeState = merger.merge(); MergeState mergeState = merger.merge();
merge.info.docCount = mergeState.mergedDocCount; merge.info.docCount = mergeState.mergedDocCount;
merge.info.setFiles(new HashSet<String>(dirWrapper.getCreatedFiles()));
// Record which codec was used to write the segment // Record which codec was used to write the segment
@ -3501,10 +3530,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (useCompoundFile) { if (useCompoundFile) {
success = false; success = false;
final String compoundFileName = IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
Collection<String> filesToRemove = merge.info.files();
try { try {
createCompoundFile(infoStream, directory, compoundFileName, checkAbort, merge.info, context); filesToRemove = createCompoundFile(infoStream, directory, checkAbort, merge.info, context);
success = true; success = true;
} catch (IOException ioe) { } catch (IOException ioe) {
synchronized(this) { synchronized(this) {
@ -3525,7 +3555,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
} }
synchronized(this) { synchronized(this) {
deleter.deleteFile(compoundFileName); deleter.deleteFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
deleter.deleteFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); deleter.deleteFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
deleter.deleteNewFiles(merge.info.files()); deleter.deleteNewFiles(merge.info.files());
} }
@ -3539,13 +3569,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// delete new non cfs files directly: they were never // delete new non cfs files directly: they were never
// registered with IFD // registered with IFD
deleter.deleteNewFiles(merge.info.files()); deleter.deleteNewFiles(filesToRemove);
if (merge.isAborted()) { if (merge.isAborted()) {
if (infoStream.isEnabled("IW")) { if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "abort merge after building CFS"); infoStream.message("IW", "abort merge after building CFS");
} }
deleter.deleteFile(compoundFileName); deleter.deleteFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
deleter.deleteFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
return 0; return 0;
} }
} }
@ -3565,7 +3596,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (!success2) { if (!success2) {
synchronized(this) { synchronized(this) {
deleter.deleteNewFiles(merge.info.files()); deleter.deleteNewFiles(merge.info.files());
} }
} }
} }
merge.info.clearFilesCache(); merge.info.clearFilesCache();
@ -3980,9 +4011,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* deletion files, this SegmentInfo must not reference such files when this * deletion files, this SegmentInfo must not reference such files when this
* method is called, because they are not allowed within a compound file. * method is called, because they are not allowed within a compound file.
*/ */
static final Collection<String> createCompoundFile(InfoStream infoStream, Directory directory, String fileName, CheckAbort checkAbort, final SegmentInfo info, IOContext context) static final Collection<String> createCompoundFile(InfoStream infoStream, Directory directory, CheckAbort checkAbort, final SegmentInfo info, IOContext context)
throws IOException { throws IOException {
final String fileName = IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
if (infoStream.isEnabled("IW")) { if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "create compound file " + fileName); infoStream.message("IW", "create compound file " + fileName);
} }
@ -4002,6 +4034,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
IOUtils.closeWhileHandlingException(prior, cfsDir); IOUtils.closeWhileHandlingException(prior, cfsDir);
} }
Set<String> siFiles = new HashSet<String>();
siFiles.add(fileName);
siFiles.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
info.setFiles(siFiles);
return files; return files;
} }
} }

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -30,6 +31,7 @@ import java.util.regex.Pattern;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Constants; import org.apache.lucene.util.Constants;
/** /**
@ -106,6 +108,19 @@ public class SegmentInfo implements Cloneable {
// nocommit why do we have this wimpy ctor...? // nocommit why do we have this wimpy ctor...?
public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
Codec codec) { Codec codec) {
// nocommit
/*
this.name = name;
this.docCount = docCount;
this.dir = dir;
delGen = NO;
this.isCompoundFile = isCompoundFile;
this.docStoreOffset = -1;
this.docStoreSegment = name;
this.codec = codec;
delCount = 0;
version = Constants.LUCENE_MAIN_VERSION;
*/
this(dir, Constants.LUCENE_MAIN_VERSION, name, docCount, -1, name, false, null, isCompoundFile, 0, codec, new HashMap<String,String>()); this(dir, Constants.LUCENE_MAIN_VERSION, name, docCount, -1, name, false, null, isCompoundFile, 0, codec, new HashMap<String,String>());
} }
@ -125,6 +140,7 @@ public class SegmentInfo implements Cloneable {
public SegmentInfo(Directory dir, String version, String name, int docCount, int docStoreOffset, public SegmentInfo(Directory dir, String version, String name, int docCount, int docStoreOffset,
String docStoreSegment, boolean docStoreIsCompoundFile, Map<Integer,Long> normGen, boolean isCompoundFile, String docStoreSegment, boolean docStoreIsCompoundFile, Map<Integer,Long> normGen, boolean isCompoundFile,
int delCount, Codec codec, Map<String,String> diagnostics) { int delCount, Codec codec, Map<String,String> diagnostics) {
assert !(dir instanceof TrackingDirectoryWrapper);
this.dir = dir; this.dir = dir;
this.version = version; this.version = version;
this.name = name; this.name = name;
@ -197,9 +213,19 @@ public class SegmentInfo implements Cloneable {
clonedNormGen = null; clonedNormGen = null;
} }
SegmentInfo newInfo = new SegmentInfo(dir, version, name, docCount, docStoreOffset, SegmentInfo newInfo = new SegmentInfo(dir, version, name, docCount, docStoreOffset,
docStoreSegment, docStoreIsCompoundFile, clonedNormGen, isCompoundFile, docStoreSegment, docStoreIsCompoundFile, clonedNormGen, isCompoundFile,
delCount, codec, new HashMap<String,String>(diagnostics)); delCount, codec, new HashMap<String,String>(diagnostics));
final Set<String> clonedFiles;
if (setFiles != null) {
clonedFiles = new HashSet<String>(setFiles);
} else {
clonedFiles = null;
}
newInfo.setFiles(clonedFiles);
newInfo.setDelGen(delGen); newInfo.setDelGen(delGen);
return newInfo; return newInfo;
} }
@ -347,20 +373,26 @@ public class SegmentInfo implements Cloneable {
* modify it. * modify it.
*/ */
public List<String> files() throws IOException { public Collection<String> files() throws IOException {
if (files == null) { // nocommit make sure when we are called we really have
// nocommit can we remove this again....? // files set ...
final Set<String> fileSet = new HashSet<String>(); if (setFiles == null) {
codec.files(this, fileSet); throw new IllegalStateException("files were not computed yet");
files = findMatchingFiles(name, dir, fileSet);
} }
return files;
Set<String> files = new HashSet<String>(setFiles);
// nocommit make this take list instead...?
// Must separately add any live docs files:
codec.liveDocsFormat().files(this, files);
return new ArrayList<String>(files);
} }
/* Called whenever any change is made that affects which /* Called whenever any change is made that affects which
* files this segment has. */ * files this segment has. */
// nocommit make private again // nocommit make private again
void clearFilesCache() { public void clearFilesCache() {
sizeInBytes = -1; sizeInBytes = -1;
files = null; files = null;
} }
@ -468,4 +500,29 @@ public class SegmentInfo implements Cloneable {
public Map<Integer,Long> getNormGen() { public Map<Integer,Long> getNormGen() {
return normGen; return normGen;
} }
private Set<String> setFiles;
// nocommit now on building a CFS we erase the files that
// are in it... maybe we should somehow preserve it...
public void setFiles(Set<String> files) {
//System.out.println("set files=" + files);
//if (files.size() == 0) {
//new Throwable().printStackTrace(System.out);
//}
setFiles = files;
}
public Set<String> getFiles() {
return setFiles;
}
public Set<String> getFiles2() throws IOException {
Set<String> files = new HashSet<String>(setFiles);
// nocommit make this take list instead...?
// Must separately add any live docs files:
codec.liveDocsFormat().files(this, files);
return files;
}
} }

View File

@ -357,7 +357,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
if (version == null || version.startsWith("3.")) { if (version == null || version.startsWith("3.")) {
String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfosFormat.SI_EXTENSION); String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfosFormat.SI_EXTENSION);
if (!directory.fileExists(fileName)) { if (!directory.fileExists(fileName)) {
//System.out.println("write 3x info seg=" + si.name + " version=" + si.getVersion() + " codec=" + si.getCodec().getName());
upgradedSIFiles.add(write3xInfo(directory, si, IOContext.DEFAULT)); upgradedSIFiles.add(write3xInfo(directory, si, IOContext.DEFAULT));
si.clearFilesCache(); si.clearFilesCache();
} }
@ -396,6 +395,8 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
// NOTE: this is NOT how 3.x is really written... // NOTE: this is NOT how 3.x is really written...
String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfosFormat.SI_EXTENSION); String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfosFormat.SI_EXTENSION);
si.getFiles().add(fileName);
//System.out.println("UPGRADE write " + fileName); //System.out.println("UPGRADE write " + fileName);
boolean success = false; boolean success = false;
IndexOutput output = dir.createOutput(fileName, context); IndexOutput output = dir.createOutput(fileName, context);
@ -430,11 +431,12 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
output.writeInt(si.getDelCount()); output.writeInt(si.getDelCount());
// hasProx: // hasProx (lie):
output.writeByte((byte) 1); output.writeByte((byte) 1);
output.writeStringStringMap(si.getDiagnostics()); output.writeStringStringMap(si.getDiagnostics());
// hasVectors: // hasVectors (lie):
output.writeByte((byte) 1); output.writeByte((byte) 1);
output.writeStringSet(si.getFiles());
success = true; success = true;
} finally { } finally {

View File

@ -19,7 +19,9 @@ package org.apache.lucene.store;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
@ -201,6 +203,8 @@ public abstract class DataInput implements Cloneable {
return clone; return clone;
} }
/** Reads a Map&lt;String,String&gt; previously written
* with {@link DataOutput#writeStringStringMap}. */
public Map<String,String> readStringStringMap() throws IOException { public Map<String,String> readStringStringMap() throws IOException {
final Map<String,String> map = new HashMap<String,String>(); final Map<String,String> map = new HashMap<String,String>();
final int count = readInt(); final int count = readInt();
@ -212,4 +216,16 @@ public abstract class DataInput implements Cloneable {
return map; return map;
} }
/** Reads a Set&lt;String&gt; previously written
* with {@link DataOutput#writeStringSet}. */
public Set<String> readStringSet() throws IOException {
final Set<String> set = new HashSet<String>();
final int count = readInt();
for(int i=0;i<count;i++) {
set.add(readString());
}
return set;
}
} }

View File

@ -19,6 +19,7 @@ package org.apache.lucene.store;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.UnicodeUtil; import org.apache.lucene.util.UnicodeUtil;
@ -273,4 +274,24 @@ public abstract class DataOutput {
} }
} }
} }
/**
* Writes a String set.
* <p>
* First the size is written as an {@link #writeInt(int) Int32},
* followed by each value written as a
* {@link #writeString(String) String}.
*
* @param set Input set. May be null (equivalent to an empty set)
*/
public void writeStringSet(Set<String> set) throws IOException {
if (set == null) {
writeInt(0);
} else {
writeInt(set.size());
for(String value : set) {
writeString(value);
}
}
}
} }

View File

@ -118,6 +118,8 @@ public final class TrackingDirectoryWrapper extends Directory implements Closeab
return other.createSlicer(name, context); return other.createSlicer(name, context);
} }
// maybe clone before returning.... all callers are
// cloning anyway....
public Set<String> getCreatedFiles() { public Set<String> getCreatedFiles() {
return createdFileNames; return createdFileNames;
} }

View File

@ -23,7 +23,9 @@ import java.io.IOException;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.io.StringWriter; import java.io.StringWriter;
import java.util.Collection; import java.util.Collection;
import java.util.HashSet;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
@ -33,6 +35,7 @@ import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Constants; import org.apache.lucene.util.Constants;
import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
@ -193,7 +196,8 @@ public class TestDoc extends LuceneTestCase {
SegmentReader r2 = new SegmentReader(si2, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, context); SegmentReader r2 = new SegmentReader(si2, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
final Codec codec = Codec.getDefault(); final Codec codec = Codec.getDefault();
SegmentMerger merger = new SegmentMerger(InfoStream.getDefault(), si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, MergeState.CheckAbort.NONE, null, new FieldInfos.Builder(), codec, context); TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.dir);
SegmentMerger merger = new SegmentMerger(InfoStream.getDefault(), trackingDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, MergeState.CheckAbort.NONE, null, new FieldInfos.Builder(), codec, context);
merger.add(r1); merger.add(r1);
merger.add(r2); merger.add(r2);
@ -203,9 +207,10 @@ public class TestDoc extends LuceneTestCase {
final SegmentInfo info = new SegmentInfo(si1.dir, Constants.LUCENE_MAIN_VERSION, merged, final SegmentInfo info = new SegmentInfo(si1.dir, Constants.LUCENE_MAIN_VERSION, merged,
si1.docCount + si2.docCount, -1, merged, si1.docCount + si2.docCount, -1, merged,
false, null, false, 0, codec, null); false, null, false, 0, codec, null);
info.setFiles(new HashSet<String>(trackingDir.getCreatedFiles()));
if (useCompoundFile) { if (useCompoundFile) {
Collection<String> filesToDelete = IndexWriter.createCompoundFile(InfoStream.getDefault(), dir, merged + ".cfs", MergeState.CheckAbort.NONE, info, newIOContext(random())); Collection<String> filesToDelete = IndexWriter.createCompoundFile(InfoStream.getDefault(), dir, MergeState.CheckAbort.NONE, info, newIOContext(random()));
info.setUseCompoundFile(true); info.setUseCompoundFile(true);
for (final String fileToDelete : filesToDelete) { for (final String fileToDelete : filesToDelete) {
si1.dir.deleteFile(fileToDelete); si1.dir.deleteFile(fileToDelete);

View File

@ -1548,10 +1548,13 @@ public class TestIndexWriter extends LuceneTestCase {
for (IndexReader r : r0.getSequentialSubReaders()) { for (IndexReader r : r0.getSequentialSubReaders()) {
SegmentInfo s = ((SegmentReader) r).getSegmentInfo(); SegmentInfo s = ((SegmentReader) r).getSegmentInfo();
assertFalse(((SegmentReader) r).getFieldInfos().hasVectors()); assertFalse(((SegmentReader) r).getFieldInfos().hasVectors());
// nocommit
/*
Set<String> files = new HashSet<String>(); Set<String> files = new HashSet<String>();
s.getCodec().termVectorsFormat().files(s, files); s.getCodec().termVectorsFormat().files(s, files);
List<String> filesExisting = SegmentInfo.findMatchingFiles(s.name, dir, files); List<String> filesExisting = SegmentInfo.findMatchingFiles(s.name, dir, files);
assertTrue(filesExisting.isEmpty()); assertTrue(filesExisting.isEmpty());
*/
} }
r0.close(); r0.close();

View File

@ -1124,49 +1124,49 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
// files and make sure we get an IOException trying to // files and make sure we get an IOException trying to
// open the index: // open the index:
public void testSimulatedCorruptIndex2() throws IOException { public void testSimulatedCorruptIndex2() throws IOException {
MockDirectoryWrapper dir = newDirectory(); MockDirectoryWrapper dir = newDirectory();
dir.setCheckIndexOnClose(false); // we are corrupting it! dir.setCheckIndexOnClose(false); // we are corrupting it!
IndexWriter writer = null; IndexWriter writer = null;
writer = new IndexWriter( writer = new IndexWriter(
dir, dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())). newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
setMergePolicy(newLogMergePolicy(true)) setMergePolicy(newLogMergePolicy(true))
); );
((LogMergePolicy) writer.getConfig().getMergePolicy()).setNoCFSRatio(1.0); ((LogMergePolicy) writer.getConfig().getMergePolicy()).setNoCFSRatio(1.0);
// add 100 documents // add 100 documents
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
addDoc(writer); addDoc(writer);
}
// close
writer.close();
long gen = SegmentInfos.getLastCommitGeneration(dir);
assertTrue("segment generation should be > 0 but got " + gen, gen > 0);
String[] files = dir.listAll();
boolean corrupted = false;
for(int i=0;i<files.length;i++) {
if (files[i].endsWith(".cfs")) {
dir.deleteFile(files[i]);
corrupted = true;
break;
} }
}
assertTrue("failed to find cfs file to remove", corrupted);
// close IndexReader reader = null;
writer.close(); try {
reader = IndexReader.open(dir);
long gen = SegmentInfos.getLastCommitGeneration(dir); fail("reader did not hit IOException on opening a corrupt index");
assertTrue("segment generation should be > 0 but got " + gen, gen > 0); } catch (Exception e) {
}
String[] files = dir.listAll(); if (reader != null) {
boolean corrupted = false; reader.close();
for(int i=0;i<files.length;i++) { }
if (files[i].endsWith(".cfs")) { dir.close();
dir.deleteFile(files[i]);
corrupted = true;
break;
}
}
assertTrue("failed to find cfs file to remove", corrupted);
IndexReader reader = null;
try {
reader = IndexReader.open(dir);
fail("reader did not hit IOException on opening a corrupt index");
} catch (Exception e) {
}
if (reader != null) {
reader.close();
}
dir.close();
} }
// Simulate a writer that crashed while writing segments // Simulate a writer that crashed while writing segments

View File

@ -189,12 +189,16 @@ public class TestTermVectorsReader extends LuceneTestCase {
for (IndexReader r : reader.getSequentialSubReaders()) { for (IndexReader r : reader.getSequentialSubReaders()) {
SegmentInfo s = ((SegmentReader) r).getSegmentInfo(); SegmentInfo s = ((SegmentReader) r).getSegmentInfo();
assertTrue(((SegmentReader) r).getFieldInfos().hasVectors()); assertTrue(((SegmentReader) r).getFieldInfos().hasVectors());
// nocommit
/*
Set<String> files = new HashSet<String>(); Set<String> files = new HashSet<String>();
s.getCodec().termVectorsFormat().files(s, files); s.getCodec().termVectorsFormat().files(s, files);
assertFalse(files.isEmpty()); assertFalse(files.isEmpty());
for (String file : files) { for (String file : files) {
assertTrue(dir.fileExists(file)); assertTrue(dir.fileExists(file));
} }
*/
} }
reader.close(); reader.close();
} }

View File

@ -24,6 +24,7 @@ import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.text.DecimalFormat; import java.text.DecimalFormat;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.List; import java.util.List;
import org.apache.lucene.index.IndexWriter; // Required for javadocs import org.apache.lucene.index.IndexWriter; // Required for javadocs
@ -148,7 +149,7 @@ public class IndexSplitter {
info.getDelCount(), info.getCodec(), info.getDiagnostics()); info.getDelCount(), info.getCodec(), info.getDiagnostics());
destInfos.add(newInfo); destInfos.add(newInfo);
// now copy files over // now copy files over
List<String> files = info.files(); Collection<String> files = info.files();
for (final String srcName : files) { for (final String srcName : files) {
File srcFile = new File(dir, srcName); File srcFile = new File(dir, srcName);
File destFile = new File(destDir, srcName); File destFile = new File(destDir, srcName);

View File

@ -109,19 +109,4 @@ public class PreFlexRWCodec extends Lucene3xCodec {
return super.storedFieldsFormat(); 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);
}
} }

View File

@ -134,11 +134,4 @@ public class Lucene40WithOrds extends PostingsFormat {
/** Extension of prox postings file */ /** Extension of prox postings file */
static final String PROX_EXTENSION = "prx"; static final String PROX_EXTENSION = "prx";
@Override
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);
}
} }

View File

@ -199,11 +199,4 @@ public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
} }
} }
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
SepPostingsReader.files(segmentInfo, segmentSuffix, files);
BlockTermsReader.files(segmentInfo, segmentSuffix, files);
FixedGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
}
} }

View File

@ -222,11 +222,4 @@ public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
} }
} }
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
SepPostingsReader.files(segmentInfo, segmentSuffix, files);
BlockTermsReader.files(segmentInfo, segmentSuffix, files);
FixedGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
}
} }

View File

@ -411,25 +411,4 @@ public class MockRandomPostingsFormat extends PostingsFormat {
return fields; return fields;
} }
@Override
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(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 (!segmentInfo.dir.fileExists(file)) {
it.remove();
}
}
//System.out.println("MockRandom.files return " + files);
}
} }

View File

@ -45,9 +45,4 @@ public class MockSepDocValuesFormat extends DocValuesFormat {
public PerDocProducer docsProducer(SegmentReadState state) throws IOException { public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new SepDocValuesProducer(state); return new SepDocValuesProducer(state);
} }
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
SepDocValuesConsumer.files(info, files);
}
} }

View File

@ -126,11 +126,4 @@ public class MockSepPostingsFormat extends PostingsFormat {
} }
} }
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
SepPostingsReader.files(segmentInfo, segmentSuffix, files);
BlockTermsReader.files(segmentInfo, segmentSuffix, files);
FixedGapTermsIndexReader.files(segmentInfo, segmentSuffix, files);
}
} }

View File

@ -89,10 +89,4 @@ public class NestedPulsingPostingsFormat extends PostingsFormat {
} }
} }
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
BlockTreeTermsReader.files(segmentInfo, segmentSuffix, files);
}
} }

View File

@ -589,10 +589,4 @@ public class RAMOnlyPostingsFormat extends PostingsFormat {
return state.get(id); return state.get(id);
} }
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) {
final String idFileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, ID_EXTENSION);
files.add(idFileName);
}
} }