LUCENE-4055: Refactor SegmentInfo and FieldInfo to make them extensible

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1343365 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2012-05-28 19:28:10 +00:00
commit e9bb487252
232 changed files with 3688 additions and 4606 deletions

View File

@ -1,3 +1,4 @@
Lucene Change Log Lucene Change Log
For more information on past and future Lucene versions, please see: For more information on past and future Lucene versions, please see:
@ -271,6 +272,8 @@ Changes in backwards compatibility policy
that take two booleans indicating whether hit scores and max that take two booleans indicating whether hit scores and max
score should be computed. (Mike McCandless) score should be computed. (Mike McCandless)
* LUCENE-4055: You can't put foreign files into the index dir anymore.
Changes in Runtime Behavior Changes in Runtime Behavior
* LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you * LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you

View File

@ -135,7 +135,7 @@ public class BlockTermsReader extends FieldsProducer {
assert numTerms >= 0; assert numTerms >= 0;
final long termsStartPointer = in.readVLong(); final long termsStartPointer = in.readVLong();
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field); final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
final long sumTotalTermFreq = fieldInfo.indexOptions == IndexOptions.DOCS_ONLY ? -1 : in.readVLong(); final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
final long sumDocFreq = in.readVLong(); final long sumDocFreq = in.readVLong();
final int docCount = in.readVInt(); final int docCount = in.readVInt();
assert !fields.containsKey(fieldInfo.name); assert !fields.containsKey(fieldInfo.name);
@ -186,10 +186,6 @@ public class BlockTermsReader extends FieldsProducer {
} }
} }
public static void files(SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockTermsWriter.TERMS_EXTENSION));
}
@Override @Override
public FieldsEnum iterator() { public FieldsEnum iterator() {
return new TermFieldsEnum(); return new TermFieldsEnum();
@ -699,13 +695,13 @@ public class BlockTermsReader extends FieldsProducer {
@Override @Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException { public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
if (fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
// Positions were not indexed: // Positions were not indexed:
return null; return null;
} }
if (needsOffsets && if (needsOffsets &&
fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) { fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
// Offsets were not indexed: // Offsets were not indexed:
return null; return null;
} }
@ -860,7 +856,7 @@ public class BlockTermsReader extends FieldsProducer {
// just skipN here: // just skipN here:
state.docFreq = freqReader.readVInt(); state.docFreq = freqReader.readVInt();
//System.out.println(" dF=" + state.docFreq); //System.out.println(" dF=" + state.docFreq);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
state.totalTermFreq = state.docFreq + freqReader.readVLong(); state.totalTermFreq = state.docFreq + freqReader.readVLong();
//System.out.println(" totTF=" + state.totalTermFreq); //System.out.println(" totTF=" + state.totalTermFreq);
} }

View File

@ -71,7 +71,7 @@ public class BlockTermsWriter extends FieldsConsumer {
public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter, public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
SegmentWriteState state, PostingsWriterBase postingsWriter) SegmentWriteState state, PostingsWriterBase postingsWriter)
throws IOException { throws IOException {
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_EXTENSION); final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
this.termsIndexWriter = termsIndexWriter; this.termsIndexWriter = termsIndexWriter;
out = state.directory.createOutput(termsFileName, state.context); out = state.directory.createOutput(termsFileName, state.context);
boolean success = false; boolean success = false;
@ -130,7 +130,7 @@ public class BlockTermsWriter extends FieldsConsumer {
out.writeVInt(field.fieldInfo.number); out.writeVInt(field.fieldInfo.number);
out.writeVLong(field.numTerms); out.writeVLong(field.numTerms);
out.writeVLong(field.termsStartPointer); out.writeVLong(field.termsStartPointer);
if (field.fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
out.writeVLong(field.sumTotalTermFreq); out.writeVLong(field.sumTotalTermFreq);
} }
out.writeVLong(field.sumDocFreq); out.writeVLong(field.sumDocFreq);
@ -302,7 +302,7 @@ public class BlockTermsWriter extends FieldsConsumer {
final TermStats stats = pendingTerms[termCount].stats; final TermStats stats = pendingTerms[termCount].stats;
assert stats != null; assert stats != null;
bytesWriter.writeVInt(stats.docFreq); bytesWriter.writeVInt(stats.docFreq);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
bytesWriter.writeVLong(stats.totalTermFreq-stats.docFreq); bytesWriter.writeVLong(stats.totalTermFreq-stats.docFreq);
} }
} }

View File

@ -20,7 +20,6 @@ package org.apache.lucene.codecs;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
import java.util.Collection;
import java.util.Comparator; import java.util.Comparator;
import java.util.Iterator; import java.util.Iterator;
import java.util.TreeMap; import java.util.TreeMap;
@ -32,7 +31,6 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum; import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState; import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms; import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum;
@ -147,7 +145,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
rootCode.length = numBytes; rootCode.length = numBytes;
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field); final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
assert fieldInfo != null: "field=" + field; assert fieldInfo != null: "field=" + field;
final long sumTotalTermFreq = fieldInfo.indexOptions == IndexOptions.DOCS_ONLY ? -1 : in.readVLong(); final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
final long sumDocFreq = in.readVLong(); final long sumDocFreq = in.readVLong();
final int docCount = in.readVInt(); final int docCount = in.readVInt();
final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0; final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0;
@ -199,11 +197,6 @@ public class BlockTreeTermsReader extends FieldsProducer {
} }
} }
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));
}
@Override @Override
public FieldsEnum iterator() { public FieldsEnum iterator() {
return new TermFieldsEnum(); return new TermFieldsEnum();
@ -732,7 +725,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
// just skipN here: // just skipN here:
termState.docFreq = statsReader.readVInt(); termState.docFreq = statsReader.readVInt();
//if (DEBUG) System.out.println(" dF=" + state.docFreq); //if (DEBUG) System.out.println(" dF=" + state.docFreq);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
termState.totalTermFreq = termState.docFreq + statsReader.readVLong(); termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
//if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq); //if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq);
} }
@ -897,13 +890,13 @@ public class BlockTreeTermsReader extends FieldsProducer {
@Override @Override
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException { public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
if (fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
// Positions were not indexed: // Positions were not indexed:
return null; return null;
} }
if (needsOffsets && if (needsOffsets &&
fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) { fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
// Offsets were not indexed: // Offsets were not indexed:
return null; return null;
} }
@ -2129,13 +2122,13 @@ public class BlockTreeTermsReader extends FieldsProducer {
@Override @Override
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException { public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
if (fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
// Positions were not indexed: // Positions were not indexed:
return null; return null;
} }
if (needsOffsets && if (needsOffsets &&
fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) { fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
// Offsets were not indexed: // Offsets were not indexed:
return null; return null;
} }
@ -2546,7 +2539,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
// just skipN here: // just skipN here:
state.docFreq = statsReader.readVInt(); state.docFreq = statsReader.readVInt();
//if (DEBUG) System.out.println(" dF=" + state.docFreq); //if (DEBUG) System.out.println(" dF=" + state.docFreq);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
state.totalTermFreq = state.docFreq + statsReader.readVLong(); state.totalTermFreq = state.docFreq + statsReader.readVLong();
//if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq); //if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq);
} }

View File

@ -144,7 +144,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
throw new IllegalArgumentException("maxItemsInBlock must be at least 2*(minItemsInBlock-1); got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock); throw new IllegalArgumentException("maxItemsInBlock must be at least 2*(minItemsInBlock-1); got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock);
} }
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_EXTENSION); final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
out = state.directory.createOutput(termsFileName, state.context); out = state.directory.createOutput(termsFileName, state.context);
boolean success = false; boolean success = false;
IndexOutput indexOut = null; IndexOutput indexOut = null;
@ -156,7 +156,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
//DEBUG = state.segmentName.equals("_4a"); //DEBUG = state.segmentName.equals("_4a");
final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_INDEX_EXTENSION); final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
indexOut = state.directory.createOutput(termsIndexFileName, state.context); indexOut = state.directory.createOutput(termsIndexFileName, state.context);
writeIndexHeader(indexOut); writeIndexHeader(indexOut);
@ -724,7 +724,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
// Write term stats, to separate byte[] blob: // Write term stats, to separate byte[] blob:
bytesWriter2.writeVInt(term.stats.docFreq); bytesWriter2.writeVInt(term.stats.docFreq);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
assert term.stats.totalTermFreq >= term.stats.docFreq; assert term.stats.totalTermFreq >= term.stats.docFreq;
bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq); bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq);
} }
@ -750,7 +750,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
// Write term stats, to separate byte[] blob: // Write term stats, to separate byte[] blob:
bytesWriter2.writeVInt(term.stats.docFreq); bytesWriter2.writeVInt(term.stats.docFreq);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
assert term.stats.totalTermFreq >= term.stats.docFreq; assert term.stats.totalTermFreq >= term.stats.docFreq;
bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq); bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq);
} }
@ -930,7 +930,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
assert rootCode != null: "field=" + field.fieldInfo.name + " numTerms=" + field.numTerms; assert rootCode != null: "field=" + field.fieldInfo.name + " numTerms=" + field.numTerms;
out.writeVInt(rootCode.length); out.writeVInt(rootCode.length);
out.writeBytes(rootCode.bytes, rootCode.offset, rootCode.length); out.writeBytes(rootCode.bytes, rootCode.offset, rootCode.length);
if (field.fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
out.writeVLong(field.sumTotalTermFreq); out.writeVLong(field.sumTotalTermFreq);
} }
out.writeVLong(field.sumDocFreq); out.writeVLong(field.sumDocFreq);

View File

@ -17,13 +17,10 @@ package org.apache.lucene.codecs;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
import java.util.Set; import java.util.Set;
import java.util.ServiceLoader; // javadocs import java.util.ServiceLoader; // javadocs
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriterConfig; // javadocs import org.apache.lucene.index.IndexWriterConfig; // javadocs
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.util.NamedSPILoader; import org.apache.lucene.util.NamedSPILoader;
/** /**
@ -45,36 +42,16 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
private final String name; private final String name;
public Codec(String name) { public Codec(String name) {
NamedSPILoader.checkServiceName(name);
this.name = name; this.name = name;
} }
/** Returns this codec's name */ /** Returns this codec's name */
@Override @Override
public String getName() { public final String getName() {
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);
}
// never inside CFS
liveDocsFormat().files(info, files);
}
/** Encodes/decodes postings */ /** Encodes/decodes postings */
public abstract PostingsFormat postingsFormat(); public abstract PostingsFormat postingsFormat();
@ -90,8 +67,8 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
/** Encodes/decodes field infos file */ /** Encodes/decodes field infos file */
public abstract FieldInfosFormat fieldInfosFormat(); public abstract FieldInfosFormat fieldInfosFormat();
/** Encodes/decodes segments file */ /** Encodes/decodes segment info file */
public abstract SegmentInfosFormat segmentInfosFormat(); public abstract SegmentInfoFormat segmentInfoFormat();
/** Encodes/decodes document normalization values */ /** Encodes/decodes document normalization values */
public abstract NormsFormat normsFormat(); public abstract NormsFormat normsFormat();

View File

@ -103,7 +103,7 @@ public abstract class DocValuesConsumer {
} }
// only finish if no exception is thrown! // only finish if no exception is thrown!
if (hasMerged) { if (hasMerged) {
finish(mergeState.mergedDocCount); finish(mergeState.segmentInfo.getDocCount());
} }
} }

View File

@ -18,11 +18,9 @@ package org.apache.lucene.codecs;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.DocValues; // javadocs import org.apache.lucene.index.DocValues; // javadocs
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
/** /**
@ -30,7 +28,10 @@ import org.apache.lucene.index.SegmentReadState;
* @lucene.experimental * @lucene.experimental
*/ */
public abstract class DocValuesFormat { public abstract class DocValuesFormat {
/** Consumes (writes) doc values during indexing. */
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException; public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
/** Produces (reads) doc values during reading/searching. */
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException; public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
} }

View File

@ -18,17 +18,19 @@ package org.apache.lucene.codecs;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.FieldInfos; // javadocs import org.apache.lucene.index.FieldInfos; // javadocs
import org.apache.lucene.index.SegmentInfo;
/** /**
* Encodes/decodes {@link FieldInfos} * Encodes/decodes {@link FieldInfos}
* @lucene.experimental * @lucene.experimental
*/ */
public abstract class FieldInfosFormat { public abstract class FieldInfosFormat {
/** Returns a {@link FieldInfosReader} to read field infos
* from the index */
public abstract FieldInfosReader getFieldInfosReader() throws IOException; public abstract FieldInfosReader getFieldInfosReader() throws IOException;
/** Returns a {@link FieldInfosWriter} to write field infos
* to the index */
public abstract FieldInfosWriter getFieldInfosWriter() throws IOException; public abstract FieldInfosWriter getFieldInfosWriter() throws IOException;
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
} }

View File

@ -22,7 +22,6 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexInput;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil; import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
@ -30,7 +29,6 @@ import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts; import org.apache.lucene.util.packed.PackedInts;
import java.util.HashMap; import java.util.HashMap;
import java.util.Collection;
import java.util.Comparator; import java.util.Comparator;
import java.io.IOException; import java.io.IOException;
@ -389,10 +387,6 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
} }
} }
public static void files(SegmentInfo info, String segmentSuffix, Collection<String> files) {
files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
}
@Override @Override
public void close() throws IOException { public void close() throws IOException {
if (in != null && !indexLoaded) { if (in != null && !indexLoaded) {

View File

@ -57,7 +57,7 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
@SuppressWarnings("unused") private final FieldInfos fieldInfos; // unread @SuppressWarnings("unused") private final FieldInfos fieldInfos; // unread
public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException { public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_INDEX_EXTENSION); final String indexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
termIndexInterval = state.termIndexInterval; termIndexInterval = state.termIndexInterval;
out = state.directory.createOutput(indexFileName, state.context); out = state.directory.createOutput(indexFileName, state.context);
boolean success = false; boolean success = false;

View File

@ -18,9 +18,9 @@ package org.apache.lucene.codecs;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set; import java.util.Collection;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfoPerCommit;
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.util.Bits; import org.apache.lucene.util.Bits;
@ -29,13 +29,20 @@ import org.apache.lucene.util.MutableBits;
/** Format for live/deleted documents /** Format for live/deleted documents
* @lucene.experimental */ * @lucene.experimental */
public abstract class LiveDocsFormat { public abstract class LiveDocsFormat {
/** creates a new mutablebits, with all bits set, for the specified size */ /** Creates a new MutableBits, with all bits set, for the specified size. */
public abstract MutableBits newLiveDocs(int size) throws IOException; public abstract MutableBits newLiveDocs(int size) throws IOException;
/** creates a new mutablebits of the same bits set and size of existing */
/** Creates a new mutablebits of the same bits set and size of existing. */
public abstract MutableBits newLiveDocs(Bits existing) throws IOException; public abstract MutableBits newLiveDocs(Bits existing) throws IOException;
/** reads bits from a file */
public abstract Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) throws IOException; /** Read live docs bits. */
/** writes bits to a file */ public abstract Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) throws IOException;
public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException;
public abstract void files(SegmentInfo info, Set<String> files) throws IOException; /** Persist live docs bits. Use {@link
* SegmentInfoPerCommit#getNextDelGen} to determine the
* generation of the deletes file you should write to. */
public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, IOContext context) throws IOException;
/** Records all files in use by this {@link SegmentInfoPerCommit} into the files argument. */
public abstract void files(SegmentInfoPerCommit info, Collection<String> files) throws IOException;
} }

View File

@ -18,17 +18,20 @@ package org.apache.lucene.codecs;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
/** /**
* format for normalization factors * format for normalization factors
*/ */
public abstract class NormsFormat { public abstract class NormsFormat {
/** Returns a {@link PerDocConsumer} to write norms to the
* index. */
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException; public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
/** Returns a {@link PerDocProducer} to read norms from the
* index. */
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException; public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
public abstract void files(SegmentInfo info, Set<String> files) throws IOException;
} }

View File

@ -18,9 +18,7 @@ package org.apache.lucene.codecs;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
@ -49,6 +47,4 @@ public abstract class PostingsBaseFormat {
public abstract PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException; public abstract PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException;
public abstract PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException; public abstract PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException;
public abstract void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException;
} }

View File

@ -70,7 +70,8 @@ public abstract class PostingsConsumer {
int df = 0; int df = 0;
long totTF = 0; long totTF = 0;
if (mergeState.fieldInfo.indexOptions == IndexOptions.DOCS_ONLY) { IndexOptions indexOptions = mergeState.fieldInfo.getIndexOptions();
if (indexOptions == IndexOptions.DOCS_ONLY) {
while(true) { while(true) {
final int doc = postings.nextDoc(); final int doc = postings.nextDoc();
if (doc == DocIdSetIterator.NO_MORE_DOCS) { if (doc == DocIdSetIterator.NO_MORE_DOCS) {
@ -82,7 +83,7 @@ public abstract class PostingsConsumer {
df++; df++;
} }
totTF = -1; totTF = -1;
} else if (mergeState.fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS) { } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
while(true) { while(true) {
final int doc = postings.nextDoc(); final int doc = postings.nextDoc();
if (doc == DocIdSetIterator.NO_MORE_DOCS) { if (doc == DocIdSetIterator.NO_MORE_DOCS) {
@ -95,7 +96,7 @@ public abstract class PostingsConsumer {
df++; df++;
totTF += freq; totTF += freq;
} }
} else if (mergeState.fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
final DocsAndPositionsEnum postingsEnum = (DocsAndPositionsEnum) postings; final DocsAndPositionsEnum postingsEnum = (DocsAndPositionsEnum) postings;
while(true) { while(true) {
final int doc = postingsEnum.nextDoc(); final int doc = postingsEnum.nextDoc();
@ -120,7 +121,7 @@ public abstract class PostingsConsumer {
df++; df++;
} }
} else { } else {
assert mergeState.fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS; assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
final DocsAndPositionsEnum postingsEnum = (DocsAndPositionsEnum) postings; final DocsAndPositionsEnum postingsEnum = (DocsAndPositionsEnum) postings;
while(true) { while(true) {
final int doc = postingsEnum.nextDoc(); final int doc = postingsEnum.nextDoc();

View File

@ -20,7 +20,6 @@ package org.apache.lucene.codecs;
import java.io.IOException; import java.io.IOException;
import java.util.Set; import java.util.Set;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.NamedSPILoader; import org.apache.lucene.util.NamedSPILoader;
@ -40,11 +39,12 @@ public abstract class PostingsFormat implements NamedSPILoader.NamedSPI {
private final String name; private final String name;
protected PostingsFormat(String name) { protected PostingsFormat(String name) {
NamedSPILoader.checkServiceName(name);
this.name = name; this.name = name;
} }
@Override @Override
public String getName() { public final String getName() {
return name; return name;
} }
@ -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 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

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.appending; package org.apache.lucene.codecs;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
@ -17,22 +17,17 @@ package org.apache.lucene.codecs.appending;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.codecs.SegmentInfosWriter; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfosFormat;
/** /**
* Append-only SegmentInfos format. * Expert: Controls the format of the
* {@link SegmentInfo} (segment metadata file).
* <p> * <p>
* Only a writer is supplied, as the format is written
* the same as {@link Lucene40SegmentInfosFormat}.
* *
* @see AppendingSegmentInfosWriter * @see SegmentInfo
* @lucene.experimental
*/ */
public class AppendingSegmentInfosFormat extends Lucene40SegmentInfosFormat { public abstract class SegmentInfoFormat {
private final SegmentInfosWriter writer = new AppendingSegmentInfosWriter(); public abstract SegmentInfoReader getSegmentInfosReader();
public abstract SegmentInfoWriter getSegmentInfosWriter();
@Override
public SegmentInfosWriter getSegmentInfosWriter() {
return writer;
}
} }

View File

@ -19,24 +19,23 @@ package org.apache.lucene.codecs;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
/** /**
* Specifies an API for classes that can read {@link SegmentInfos} information. * Specifies an API for classes that can read {@link SegmentInfo} information.
* @lucene.experimental * @lucene.experimental
*/ */
public abstract class SegmentInfosReader {
public abstract class SegmentInfoReader {
/** /**
* Read {@link SegmentInfos} data from a directory. * Read {@link SegmentInfo} data from a directory.
* @param directory directory to read from * @param directory directory to read from
* @param segmentsFileName name of the "segments_N" file * @param segmentName name of the segment to read
* @param header input of "segments_N" file after reading preamble * @return infos instance to be populated with data
* @param infos empty instance to be populated with data
* @throws IOException * @throws IOException
*/ */
public abstract void read(Directory directory, String segmentsFileName, ChecksumIndexInput header, SegmentInfos infos, IOContext context) throws IOException; public abstract SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException;
} }

View File

@ -1,6 +1,6 @@
package org.apache.lucene.codecs.appending; package org.apache.lucene.codecs;
/* /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership. * this work for additional information regarding copyright ownership.
@ -19,21 +19,21 @@ package org.apache.lucene.codecs.appending;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfosWriter; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.store.IndexOutput; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/** /**
* Append-only SegmentInfos writer. * Specifies an API for classes that can write out {@link SegmentInfo} data.
* <p> * @lucene.experimental
* Extends {@link Lucene40SegmentInfosWriter}, writing the same
* format, but the first phase of a two-phase commit
* ({@link #prepareCommit(IndexOutput)}) is not implemented.
*/ */
public class AppendingSegmentInfosWriter extends Lucene40SegmentInfosWriter {
@Override public abstract class SegmentInfoWriter {
public void prepareCommit(IndexOutput segmentOutput) throws IOException {
// noop
}
/**
* Write {@link SegmentInfo} data.
* @throws IOException
*/
public abstract void write(Directory dir, SegmentInfo info, FieldInfos fis, IOContext ioContext) throws IOException;
} }

View File

@ -1,42 +0,0 @@
package org.apache.lucene.codecs;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.index.SegmentInfos; // javadocs
/**
* Expert: Controls the format of the
* {@link SegmentInfos} (segments file).
* <p>
* NOTE: This isn't a per-segment file. If you change the format, other versions
* of lucene won't be able to read it.
*
* @see SegmentInfos
* @lucene.experimental
*/
// TODO: would be great to handle this situation better.
// ideally a custom implementation could implement two-phase commit differently,
// (e.g. atomic rename), and ideally all versions of lucene could still read it.
// but this is just reflecting reality as it is today...
//
// also, perhaps the name should change (to cover all global files like .fnx?)
// then again, maybe we can just remove that file...
public abstract class SegmentInfosFormat {
public abstract SegmentInfosReader getSegmentInfosReader();
public abstract SegmentInfosWriter getSegmentInfosWriter();
}

View File

@ -1,64 +0,0 @@
package org.apache.lucene.codecs;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
/**
* Specifies an API for classes that can write out {@link SegmentInfos} data.
* @lucene.experimental
*/
public abstract class SegmentInfosWriter {
/**
* Write {@link SegmentInfos} data without closing the output. The returned
* output will become finished only after a successful completion of
* "two phase commit" that first calls {@link #prepareCommit(IndexOutput)} and
* then {@link #finishCommit(IndexOutput)}.
* @param dir directory to write data to
* @param segmentsFileName name of the "segments_N" file to create
* @param infos data to write
* @return an instance of {@link IndexOutput} to be used in subsequent "two
* phase commit" operations as described above.
* @throws IOException
*/
public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, String codecID, SegmentInfos infos, IOContext context) throws IOException;
/**
* First phase of the two-phase commit - ensure that all output can be
* successfully written out.
* @param out an instance of {@link IndexOutput} returned from a previous
* call to {@link #writeInfos(Directory, String, String, SegmentInfos, IOContext)}.
* @throws IOException
*/
public abstract void prepareCommit(IndexOutput out) throws IOException;
/**
* Second phase of the two-phase commit. In this step the output should be
* finalized and closed.
* @param out an instance of {@link IndexOutput} returned from a previous
* call to {@link #writeInfos(Directory, String, String, SegmentInfos, IOContext)}.
* @throws IOException
*/
public abstract void finishCommit(IndexOutput out) throws IOException;
}

View File

@ -1,13 +1,5 @@
package org.apache.lucene.codecs; package org.apache.lucene.codecs;
import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
@ -25,11 +17,22 @@ import org.apache.lucene.store.IOContext;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/** /**
* Controls the format of stored fields * Controls the format of stored fields
*/ */
public abstract class StoredFieldsFormat { public abstract class StoredFieldsFormat {
/** Returns a {@link StoredFieldsReader} to load stored
* fields. */
public abstract StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException; 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(SegmentInfo info, Set<String> files) throws IOException; /** Returns a {@link StoredFieldsWriter} to write stored
* fields. */
public abstract StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException;
} }

View File

@ -1,15 +1,5 @@
package org.apache.lucene.codecs; package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.util.Bits;
/** /**
* Copyright 2004 The Apache Software Foundation * Copyright 2004 The Apache Software Foundation
* *
@ -26,6 +16,16 @@ import org.apache.lucene.util.Bits;
* the License. * the License.
*/ */
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.util.Bits;
/** /**
* Codec API for writing stored fields: * Codec API for writing stored fields:
* <p> * <p>
@ -34,7 +34,7 @@ import org.apache.lucene.util.Bits;
* informing the Codec how many fields will be written. * informing the Codec how many fields will be written.
* <li>{@link #writeField(FieldInfo, IndexableField)} is called for * <li>{@link #writeField(FieldInfo, IndexableField)} is called for
* each field in the document. * each field in the document.
* <li>After all documents have been written, {@link #finish(int)} * <li>After all documents have been written, {@link #finish(FieldInfos, int)}
* is called for verification/sanity-checks. * is called for verification/sanity-checks.
* <li>Finally the writer is closed ({@link #close()}) * <li>Finally the writer is closed ({@link #close()})
* </ol> * </ol>
@ -63,12 +63,12 @@ public abstract class StoredFieldsWriter implements Closeable {
* calls to {@link #startDocument(int)}, but a Codec should * calls to {@link #startDocument(int)}, but a Codec should
* check that this is the case to detect the JRE bug described * check that this is the case to detect the JRE bug described
* in LUCENE-1282. */ * in LUCENE-1282. */
public abstract void finish(int numDocs) throws IOException; public abstract void finish(FieldInfos fis, int numDocs) throws IOException;
/** Merges in the stored fields from the readers in /** Merges in the stored fields from the readers in
* <code>mergeState</code>. The default implementation skips * <code>mergeState</code>. The default implementation skips
* over deleted documents, and uses {@link #startDocument(int)}, * over deleted documents, and uses {@link #startDocument(int)},
* {@link #writeField(FieldInfo, IndexableField)}, and {@link #finish(int)}, * {@link #writeField(FieldInfo, IndexableField)}, and {@link #finish(FieldInfos, int)},
* returning the number of documents that were written. * returning the number of documents that were written.
* Implementations can override this method for more sophisticated * Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */ * merging (bulk-byte copying, etc). */
@ -94,7 +94,7 @@ public abstract class StoredFieldsWriter implements Closeable {
mergeState.checkAbort.work(300); mergeState.checkAbort.work(300);
} }
} }
finish(docCount); finish(mergeState.fieldInfos, docCount);
return docCount; return docCount;
} }

View File

@ -18,7 +18,6 @@ package org.apache.lucene.codecs;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfo;
@ -29,7 +28,11 @@ import org.apache.lucene.store.IOContext;
* Controls the format of term vectors * Controls the format of term vectors
*/ */
public abstract class TermVectorsFormat { public abstract class TermVectorsFormat {
/** Returns a {@link TermVectorsReader} to read term
* vectors. */
public abstract TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException; 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(SegmentInfo info, Set<String> files) throws IOException; /** Returns a {@link TermVectorsWriter} to write term
* vectors. */
public abstract TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException;
} }

View File

@ -49,7 +49,7 @@ import org.apache.lucene.util.BytesRef;
* <li>If offsets and/or positions are enabled, then * <li>If offsets and/or positions are enabled, then
* {@link #addPosition(int, int, int)} will be called for each term * {@link #addPosition(int, int, int)} will be called for each term
* occurrence. * occurrence.
* <li>After all documents have been written, {@link #finish(int)} * <li>After all documents have been written, {@link #finish(FieldInfos, int)}
* is called for verification/sanity-checks. * is called for verification/sanity-checks.
* <li>Finally the writer is closed ({@link #close()}) * <li>Finally the writer is closed ({@link #close()})
* </ol> * </ol>
@ -90,7 +90,7 @@ public abstract class TermVectorsWriter implements Closeable {
* calls to {@link #startDocument(int)}, but a Codec should * calls to {@link #startDocument(int)}, but a Codec should
* check that this is the case to detect the JRE bug described * check that this is the case to detect the JRE bug described
* in LUCENE-1282. */ * in LUCENE-1282. */
public abstract void finish(int numDocs) throws IOException; public abstract void finish(FieldInfos fis, int numDocs) throws IOException;
/** /**
* Called by IndexWriter when writing new segments. * Called by IndexWriter when writing new segments.
@ -137,7 +137,7 @@ public abstract class TermVectorsWriter implements Closeable {
* over deleted documents, and uses {@link #startDocument(int)}, * over deleted documents, and uses {@link #startDocument(int)},
* {@link #startField(FieldInfo, int, boolean, boolean)}, * {@link #startField(FieldInfo, int, boolean, boolean)},
* {@link #startTerm(BytesRef, int)}, {@link #addPosition(int, int, int)}, * {@link #startTerm(BytesRef, int)}, {@link #addPosition(int, int, int)},
* and {@link #finish(int)}, * and {@link #finish(FieldInfos, int)},
* returning the number of documents that were written. * returning the number of documents that were written.
* Implementations can override this method for more sophisticated * Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */ * merging (bulk-byte copying, etc). */
@ -159,7 +159,7 @@ public abstract class TermVectorsWriter implements Closeable {
mergeState.checkAbort.work(300); mergeState.checkAbort.work(300);
} }
} }
finish(docCount); finish(mergeState.fieldInfos, docCount);
return docCount; return docCount;
} }

View File

@ -79,9 +79,10 @@ public abstract class TermsConsumer {
long sumTotalTermFreq = 0; long sumTotalTermFreq = 0;
long sumDocFreq = 0; long sumDocFreq = 0;
long sumDFsinceLastAbortCheck = 0; long sumDFsinceLastAbortCheck = 0;
FixedBitSet visitedDocs = new FixedBitSet(mergeState.mergedDocCount); FixedBitSet visitedDocs = new FixedBitSet(mergeState.segmentInfo.getDocCount());
if (mergeState.fieldInfo.indexOptions == IndexOptions.DOCS_ONLY) { IndexOptions indexOptions = mergeState.fieldInfo.getIndexOptions();
if (indexOptions == IndexOptions.DOCS_ONLY) {
if (docsEnum == null) { if (docsEnum == null) {
docsEnum = new MappingMultiDocsEnum(); docsEnum = new MappingMultiDocsEnum();
} }
@ -109,7 +110,7 @@ public abstract class TermsConsumer {
} }
} }
} }
} else if (mergeState.fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS) { } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
if (docsAndFreqsEnum == null) { if (docsAndFreqsEnum == null) {
docsAndFreqsEnum = new MappingMultiDocsEnum(); docsAndFreqsEnum = new MappingMultiDocsEnum();
} }
@ -136,7 +137,7 @@ public abstract class TermsConsumer {
} }
} }
} }
} else if (mergeState.fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
if (postingsEnum == null) { if (postingsEnum == null) {
postingsEnum = new MappingMultiDocsAndPositionsEnum(); postingsEnum = new MappingMultiDocsAndPositionsEnum();
} }
@ -170,7 +171,7 @@ public abstract class TermsConsumer {
} }
} }
} else { } else {
assert mergeState.fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS; assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
if (postingsEnum == null) { if (postingsEnum == null) {
postingsEnum = new MappingMultiDocsAndPositionsEnum(); postingsEnum = new MappingMultiDocsAndPositionsEnum();
} }

View File

@ -21,13 +21,11 @@ import java.io.IOException;
import java.io.FileOutputStream; // for toDot import java.io.FileOutputStream; // for toDot
import java.io.OutputStreamWriter; // for toDot import java.io.OutputStreamWriter; // for toDot
import java.io.Writer; // for toDot import java.io.Writer; // for toDot
import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
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.IndexInput; import org.apache.lucene.store.IndexInput;
@ -217,10 +215,6 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
} }
} }
public static void files(SegmentInfo info, String segmentSuffix, Collection<String> files) {
files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
}
@Override @Override
public void close() throws IOException { public void close() throws IOException {
if (in != null && !indexLoaded) { if (in != null && !indexLoaded) {

View File

@ -173,7 +173,7 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
// in the extremes. // in the extremes.
public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException { public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException {
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_INDEX_EXTENSION); final String indexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
out = state.directory.createOutput(indexFileName, state.context); out = state.directory.createOutput(indexFileName, state.context);
boolean success = false; boolean success = false;
try { try {

View File

@ -23,7 +23,7 @@ import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec; import org.apache.lucene.codecs.lucene40.Lucene40Codec;
@ -31,6 +31,7 @@ import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat; import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat; import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat; import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat; import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat; import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
@ -46,7 +47,7 @@ public class AppendingCodec extends Codec {
} }
private final PostingsFormat postings = new AppendingPostingsFormat(); private final PostingsFormat postings = new AppendingPostingsFormat();
private final SegmentInfosFormat infos = new AppendingSegmentInfosFormat(); private final SegmentInfoFormat infos = new Lucene40SegmentInfoFormat();
private final StoredFieldsFormat fields = new Lucene40StoredFieldsFormat(); private final StoredFieldsFormat fields = new Lucene40StoredFieldsFormat();
private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat(); private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat();
private final TermVectorsFormat vectors = new Lucene40TermVectorsFormat(); private final TermVectorsFormat vectors = new Lucene40TermVectorsFormat();
@ -75,7 +76,7 @@ public class AppendingCodec extends Codec {
} }
@Override @Override
public SegmentInfosFormat segmentInfosFormat() { public SegmentInfoFormat segmentInfoFormat() {
return infos; return infos;
} }

View File

@ -18,9 +18,7 @@ package org.apache.lucene.codecs.appending;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter; import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
@ -29,7 +27,6 @@ import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader; import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter; import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -60,7 +57,7 @@ class AppendingPostingsFormat extends PostingsFormat {
@Override @Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
boolean success = false; boolean success = false;
try { try {
@ -80,10 +77,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

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene3x;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.HashSet;
import java.util.Set; import java.util.Set;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
@ -28,13 +29,14 @@ import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer; import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer; import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat; import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfoPerCommit;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
@ -58,7 +60,7 @@ public class Lucene3xCodec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene3xFieldInfosFormat(); private final FieldInfosFormat fieldInfosFormat = new Lucene3xFieldInfosFormat();
private final SegmentInfosFormat infosFormat = new Lucene3xSegmentInfosFormat(); private final SegmentInfoFormat infosFormat = new Lucene3xSegmentInfoFormat();
private final Lucene3xNormsFormat normsFormat = new Lucene3xNormsFormat(); private final Lucene3xNormsFormat normsFormat = new Lucene3xNormsFormat();
@ -68,7 +70,7 @@ public class Lucene3xCodec extends Codec {
// TODO: this should really be a different impl // TODO: this should really be a different impl
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat() { private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat() {
@Override @Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException { public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, 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");
} }
}; };
@ -84,9 +86,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
@ -115,7 +114,7 @@ public class Lucene3xCodec extends Codec {
} }
@Override @Override
public SegmentInfosFormat segmentInfosFormat() { public SegmentInfoFormat segmentInfoFormat() {
return infosFormat; return infosFormat;
} }
@ -129,30 +128,24 @@ public class Lucene3xCodec extends Codec {
return liveDocsFormat; return liveDocsFormat;
} }
// overrides the default implementation in codec.java to handle CFS without CFE, /** Returns file names for shared doc stores, if any, else
// shared doc stores, compound doc stores, separate norms, etc * null. */
@Override public static Set<String> getDocStoreFiles(SegmentInfo info) {
public void files(SegmentInfo info, Set<String> files) throws IOException { if (Lucene3xSegmentInfoFormat.getDocStoreOffset(info) != -1) {
if (info.getUseCompoundFile()) { final String dsName = Lucene3xSegmentInfoFormat.getDocStoreSegment(info);
files.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION)); Set<String> files = new HashSet<String>();
if (Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(info)) {
files.add(IndexFileNames.segmentFileName(dsName, "", COMPOUND_FILE_STORE_EXTENSION));
} else { } else {
postingsFormat().files(info, "", files); files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION));
storedFieldsFormat().files(info, files); files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION));
termVectorsFormat().files(info, files); files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION));
fieldInfosFormat().files(info, files); files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION));
// TODO: segmentInfosFormat should be allowed to declare additional files files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION));
// if it wants, in addition to segments_N
docValuesFormat().files(info, files);
normsFormat().files(info, files);
} }
// never inside CFS return files;
liveDocsFormat().files(info, files); } else {
((Lucene3xNormsFormat)normsFormat()).separateFiles(info, files); return null;
// shared docstores: these guys check the hair
if (info.getDocStoreOffset() != -1) {
storedFieldsFormat().files(info, files);
termVectorsFormat().files(info, files);
} }
} }
} }

View File

@ -18,12 +18,10 @@ package org.apache.lucene.codecs.lucene3x;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosReader; import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter; import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.SegmentInfo;
/** /**
* Lucene3x ReadOnly FieldInfosFromat implementation * Lucene3x ReadOnly FieldInfosFromat implementation
@ -44,9 +42,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

@ -1,4 +1,5 @@
package org.apache.lucene.codecs.lucene3x; package org.apache.lucene.codecs.lucene3x;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
@ -15,8 +16,9 @@ package org.apache.lucene.codecs.lucene3x;
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set; import java.util.Collections;
import org.apache.lucene.codecs.FieldInfosReader; import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
@ -26,7 +28,6 @@ import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException; import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
@ -59,10 +60,6 @@ class Lucene3xFieldInfosReader extends FieldInfosReader {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION); final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext); IndexInput input = directory.openInput(fileName, iocontext);
boolean hasVectors = false;
boolean hasFreq = false;
boolean hasProx = false;
try { try {
final int format = input.readVInt(); final int format = input.readVInt();
@ -103,23 +100,16 @@ class Lucene3xFieldInfosReader extends FieldInfosReader {
if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
storePayloads = false; storePayloads = false;
} }
hasVectors |= storeTermVector;
hasProx |= isIndexed && indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, null, isIndexed && !omitNorms? Type.FIXED_INTS_8 : null); omitNorms, storePayloads, indexOptions, null, isIndexed && !omitNorms? Type.FIXED_INTS_8 : null, Collections.<String,String>emptyMap());
} }
if (input.getFilePointer() != input.length()) { if (input.getFilePointer() != input.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
} }
return new FieldInfos(infos, hasFreq, hasProx, hasVectors); return new FieldInfos(infos);
} finally { } finally {
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

@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene3x;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
@ -96,10 +95,10 @@ class Lucene3xFields extends FieldsProducer {
freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION), context); freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION), context);
boolean anyProx = false; boolean anyProx = false;
for (FieldInfo fi : fieldInfos) { for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed) { if (fi.isIndexed()) {
fields.put(fi.name, fi); fields.put(fi.name, fi);
preTerms.put(fi.name, new PreTerms(fi)); preTerms.put(fi.name, new PreTerms(fi));
if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { if (fi.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
anyProx = true; anyProx = true;
} }
} }
@ -133,23 +132,6 @@ class Lucene3xFields extends FieldsProducer {
return true; return true;
} }
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));
if (info.getHasProx()) {
// LUCENE-1739: for certain versions of 2.9-dev,
// hasProx would be incorrectly computed during
// indexing as true, and then stored into the segments
// file, when it should have been false. So we do the
// extra check, here:
final String prx = IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION);
if (info.dir.fileExists(prx)) {
files.add(prx);
}
}
}
@Override @Override
public FieldsEnum iterator() throws IOException { public FieldsEnum iterator() throws IOException {
return new PreFlexFieldsEnum(); return new PreFlexFieldsEnum();
@ -952,7 +934,7 @@ class Lucene3xFields extends FieldsProducer {
@Override @Override
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException { public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
PreDocsEnum docsEnum; PreDocsEnum docsEnum;
if (needsFreqs && fieldInfo.indexOptions == IndexOptions.DOCS_ONLY) { if (needsFreqs && fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY) {
return null; return null;
} else if (reuse == null || !(reuse instanceof PreDocsEnum)) { } else if (reuse == null || !(reuse instanceof PreDocsEnum)) {
docsEnum = new PreDocsEnum(); docsEnum = new PreDocsEnum();
@ -973,7 +955,7 @@ class Lucene3xFields extends FieldsProducer {
} }
PreDocsAndPositionsEnum docsPosEnum; PreDocsAndPositionsEnum docsPosEnum;
if (fieldInfo.indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
return null; return null;
} else if (reuse == null || !(reuse instanceof PreDocsAndPositionsEnum)) { } else if (reuse == null || !(reuse instanceof PreDocsAndPositionsEnum)) {
docsPosEnum = new PreDocsAndPositionsEnum(); docsPosEnum = new PreDocsAndPositionsEnum();

View File

@ -18,13 +18,11 @@ package org.apache.lucene.codecs.lucene3x;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer; import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer; import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
/** /**
@ -36,17 +34,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

@ -23,7 +23,6 @@ import java.util.HashMap;
import java.util.IdentityHashMap; import java.util.IdentityHashMap;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.Map.Entry;
import org.apache.lucene.codecs.PerDocProducer; import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValues;
@ -69,16 +68,15 @@ class Lucene3xNormsProducer extends PerDocProducer {
// but we just don't do any seeks or reading yet. // but we just don't do any seeks or reading yet.
public Lucene3xNormsProducer(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context) throws IOException { public Lucene3xNormsProducer(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context) throws IOException {
Directory separateNormsDir = info.dir; // separate norms are never inside CFS Directory separateNormsDir = info.dir; // separate norms are never inside CFS
maxdoc = info.docCount; maxdoc = info.getDocCount();
String segmentName = info.name; String segmentName = info.name;
Map<Integer,Long> normGen = info.getNormGen();
boolean success = false; boolean success = false;
try { try {
long nextNormSeek = NORMS_HEADER.length; //skip header (header unused for now) long nextNormSeek = NORMS_HEADER.length; //skip header (header unused for now)
for (FieldInfo fi : fields) { for (FieldInfo fi : fields) {
if (fi.hasNorms()) { if (fi.hasNorms()) {
String fileName = getNormFilename(segmentName, normGen, fi.number); String fileName = getNormFilename(info, fi.number);
Directory d = hasSeparateNorms(normGen, fi.number) ? separateNormsDir : dir; Directory d = hasSeparateNorms(info, fi.number) ? separateNormsDir : dir;
// singleNormFile means multiple norms share this file // singleNormFile means multiple norms share this file
boolean singleNormFile = IndexFileNames.matchesExtension(fileName, NORMS_EXTENSION); boolean singleNormFile = IndexFileNames.matchesExtension(fileName, NORMS_EXTENSION);
@ -142,22 +140,24 @@ class Lucene3xNormsProducer extends PerDocProducer {
} }
} }
private static String getNormFilename(String segmentName, Map<Integer,Long> normGen, int number) { private static String getNormFilename(SegmentInfo info, int number) {
if (hasSeparateNorms(normGen, number)) { if (hasSeparateNorms(info, number)) {
return IndexFileNames.fileNameFromGeneration(segmentName, SEPARATE_NORMS_EXTENSION + number, normGen.get(number)); long gen = Long.parseLong(info.getAttribute(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + number));
return IndexFileNames.fileNameFromGeneration(info.name, SEPARATE_NORMS_EXTENSION + number, gen);
} else { } else {
// single file for all norms // single file for all norms
return IndexFileNames.fileNameFromGeneration(segmentName, NORMS_EXTENSION, SegmentInfo.WITHOUT_GEN); return IndexFileNames.segmentFileName(info.name, "", NORMS_EXTENSION);
} }
} }
private static boolean hasSeparateNorms(Map<Integer,Long> normGen, int number) { private static boolean hasSeparateNorms(SegmentInfo info, int number) {
if (normGen == null) { String v = info.getAttribute(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + number);
if (v == null) {
return false; return false;
} else {
assert Long.parseLong(v) != SegmentInfo.NO;
return true;
} }
Long gen = normGen.get(number);
return gen != null && gen.longValue() != SegmentInfo.NO;
} }
static final class NormSource extends Source { static final class NormSource extends Source {
@ -193,29 +193,6 @@ class Lucene3xNormsProducer extends PerDocProducer {
} }
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 (info.dir.fileExists(normsFileName)) {
// only needed to do this in 3x - 4x can decide if the norms are present
files.add(normsFileName);
}
}
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()) {
long gen = entry.getValue();
if (gen >= SegmentInfo.YES) {
// Definitely a separate norm file, with generation:
files.add(IndexFileNames.fileNameFromGeneration(info.name, SEPARATE_NORMS_EXTENSION + entry.getKey(), gen));
}
}
}
}
private class NormsDocValues extends DocValues { private class NormsDocValues extends DocValues {
private final IndexInput file; private final IndexInput file;
private final long offset; private final long offset;

View File

@ -17,19 +17,17 @@ package org.apache.lucene.codecs.lucene3x;
* limitations under the License. * limitations under the License.
*/ */
import java.util.Set;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
/** Codec that reads the pre-flex-indexing postings /** Codec that reads the pre-flex-indexing postings
* format. It does not provide a writer because newly * format. It does not provide a writer because newly
* written segments should use StandardCodec. * written segments should use the Codec configured on IndexWriter.
* *
* @deprecated (4.0) This is only used to read indexes created * @deprecated (4.0) This is only used to read indexes created
* before 4.0. * before 4.0.
@ -63,10 +61,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

@ -0,0 +1,89 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.codecs.SegmentInfoWriter;
import org.apache.lucene.index.SegmentInfo;
/**
* Lucene3x ReadOnly SegmentInfoFormat implementation
* @deprecated (4.0) This is only used to read indexes created
* before 4.0.
* @lucene.experimental
*/
@Deprecated
public class Lucene3xSegmentInfoFormat extends SegmentInfoFormat {
private final SegmentInfoReader reader = new Lucene3xSegmentInfoReader();
/** This format adds optional per-segment String
* diagnostics storage, and switches userData to Map */
public static final int FORMAT_DIAGNOSTICS = -9;
/** Each segment records whether it has term vectors */
public static final int FORMAT_HAS_VECTORS = -10;
/** Each segment records the Lucene version that created it. */
public static final int FORMAT_3_1 = -11;
/** Extension used for saving each SegmentInfo, once a 3.x
* index is first committed to with 4.0. */
public static final String UPGRADED_SI_EXTENSION = "si";
public static final String UPGRADED_SI_CODEC_NAME = "Lucene3xSegmentInfo";
public static final int UPGRADED_SI_VERSION_START = 0;
public static final int UPGRADED_SI_VERSION_CURRENT = UPGRADED_SI_VERSION_START;
@Override
public SegmentInfoReader getSegmentInfosReader() {
return reader;
}
@Override
public SegmentInfoWriter getSegmentInfosWriter() {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
// only for backwards compat
public static final String DS_OFFSET_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dsoffset";
public static final String DS_NAME_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dsname";
public static final String DS_COMPOUND_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dscompound";
public static final String NORMGEN_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".normgen";
public static final String NORMGEN_PREFIX = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".normfield";
/**
* @return if this segment shares stored fields & vectors, this
* offset is where in that file this segment's docs begin
*/
public static int getDocStoreOffset(SegmentInfo si) {
String v = si.getAttribute(DS_OFFSET_KEY);
return v == null ? -1 : Integer.parseInt(v);
}
/** @return name used to derive fields/vectors file we share with other segments */
public static String getDocStoreSegment(SegmentInfo si) {
String v = si.getAttribute(DS_NAME_KEY);
return v == null ? si.name : v;
}
/** @return whether doc store files are stored in compound file (*.cfx) */
public static boolean getDocStoreIsCompoundFile(SegmentInfo si) {
String v = si.getAttribute(DS_COMPOUND_KEY);
return v == null ? false : Boolean.parseBoolean(v);
}
}

View File

@ -0,0 +1,272 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfoPerCommit;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 3x implementation of {@link SegmentInfoReader}.
* @lucene.experimental
* @deprecated
*/
@Deprecated
public class Lucene3xSegmentInfoReader extends SegmentInfoReader {
public static void readLegacyInfos(SegmentInfos infos, Directory directory, IndexInput input, int format) throws IOException {
infos.version = input.readLong(); // read version
infos.counter = input.readInt(); // read counter
Lucene3xSegmentInfoReader reader = new Lucene3xSegmentInfoReader();
for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
SegmentInfoPerCommit siPerCommit = reader.readLegacySegmentInfo(directory, format, input);
SegmentInfo si = siPerCommit.info;
if (si.getVersion() == null) {
// Could be a 3.0 - try to open the doc stores - if it fails, it's a
// 2.x segment, and an IndexFormatTooOldException will be thrown,
// which is what we want.
Directory dir = directory;
if (Lucene3xSegmentInfoFormat.getDocStoreOffset(si) != -1) {
if (Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(si)) {
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
Lucene3xSegmentInfoFormat.getDocStoreSegment(si), "",
Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), IOContext.READONCE, false);
}
} else if (si.getUseCompoundFile()) {
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE, false);
}
try {
Lucene3xStoredFieldsReader.checkCodeVersion(dir, Lucene3xSegmentInfoFormat.getDocStoreSegment(si));
} finally {
// If we opened the directory, close it
if (dir != directory) dir.close();
}
// Above call succeeded, so it's a 3.0 segment. Upgrade it so the next
// time the segment is read, its version won't be null and we won't
// need to open FieldsReader every time for each such segment.
si.setVersion("3.0");
} else if (si.getVersion().equals("2.x")) {
// If it's a 3x index touched by 3.1+ code, then segments record their
// version, whether they are 2.x ones or not. We detect that and throw
// appropriate exception.
throw new IndexFormatTooOldException("segment " + si.name + " in resource " + input, si.getVersion());
}
infos.add(siPerCommit);
}
infos.userData = input.readStringStringMap();
}
@Override
public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
// NOTE: this is NOT how 3.x is really written...
String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene3xSegmentInfoFormat.UPGRADED_SI_EXTENSION);
boolean success = false;
IndexInput input = directory.openInput(fileName, context);
try {
SegmentInfo si = readUpgradedSegmentInfo(segmentName, directory, input);
success = true;
return si;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(input);
} else {
input.close();
}
}
}
private static void addIfExists(Directory dir, Set<String> files, String fileName) throws IOException {
if (dir.fileExists(fileName)) {
files.add(fileName);
}
}
/** reads from legacy 3.x segments_N */
private SegmentInfoPerCommit readLegacySegmentInfo(Directory dir, int format, IndexInput input) throws IOException {
// check that it is a format we can understand
if (format > Lucene3xSegmentInfoFormat.FORMAT_DIAGNOSTICS) {
throw new IndexFormatTooOldException(input, format,
Lucene3xSegmentInfoFormat.FORMAT_DIAGNOSTICS, Lucene3xSegmentInfoFormat.FORMAT_3_1);
}
if (format < Lucene3xSegmentInfoFormat.FORMAT_3_1) {
throw new IndexFormatTooNewException(input, format,
Lucene3xSegmentInfoFormat.FORMAT_DIAGNOSTICS, Lucene3xSegmentInfoFormat.FORMAT_3_1);
}
final String version;
if (format <= Lucene3xSegmentInfoFormat.FORMAT_3_1) {
version = input.readString();
} else {
version = null;
}
final String name = input.readString();
final int docCount = input.readInt();
final long delGen = input.readLong();
final int docStoreOffset = input.readInt();
final Map<String,String> attributes = new HashMap<String,String>();
// parse the docstore stuff and shove it into attributes
final String docStoreSegment;
final boolean docStoreIsCompoundFile;
if (docStoreOffset != -1) {
docStoreSegment = input.readString();
docStoreIsCompoundFile = input.readByte() == SegmentInfo.YES;
attributes.put(Lucene3xSegmentInfoFormat.DS_OFFSET_KEY, Integer.toString(docStoreOffset));
attributes.put(Lucene3xSegmentInfoFormat.DS_NAME_KEY, docStoreSegment);
attributes.put(Lucene3xSegmentInfoFormat.DS_COMPOUND_KEY, Boolean.toString(docStoreIsCompoundFile));
} else {
docStoreSegment = name;
docStoreIsCompoundFile = false;
}
// pre-4.0 indexes write a byte if there is a single norms file
byte b = input.readByte();
//System.out.println("version=" + version + " name=" + name + " docCount=" + docCount + " delGen=" + delGen + " dso=" + docStoreOffset + " dss=" + docStoreSegment + " dssCFs=" + docStoreIsCompoundFile + " b=" + b + " format=" + format);
assert 1 == b : "expected 1 but was: "+ b + " format: " + format;
final int numNormGen = input.readInt();
final Map<Integer,Long> normGen;
if (numNormGen == SegmentInfo.NO) {
normGen = null;
} else {
normGen = new HashMap<Integer, Long>();
for(int j=0;j<numNormGen;j++) {
normGen.put(j, input.readLong());
}
}
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final int delCount = input.readInt();
assert delCount <= docCount;
final boolean hasProx = input.readByte() == 1;
final Map<String,String> diagnostics = input.readStringStringMap();
if (format <= Lucene3xSegmentInfoFormat.FORMAT_HAS_VECTORS) {
// NOTE: unused
final int hasVectors = input.readByte();
}
// Replicate logic from 3.x's SegmentInfo.files():
final Set<String> files = new HashSet<String>();
if (isCompoundFile) {
files.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
} else {
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xFieldInfosReader.FIELD_INFOS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xPostingsFormat.FREQ_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xPostingsFormat.PROX_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xPostingsFormat.TERMS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xNormsProducer.NORMS_EXTENSION));
}
if (docStoreOffset != -1) {
if (docStoreIsCompoundFile) {
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION));
} else {
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION));
}
} else if (!isCompoundFile) {
files.add(IndexFileNames.segmentFileName(name, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(name, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION));
}
// parse the normgen stuff and shove it into attributes
if (normGen != null) {
attributes.put(Lucene3xSegmentInfoFormat.NORMGEN_KEY, Integer.toString(numNormGen));
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(name, "s" + ent.getKey(), gen));
attributes.put(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + ent.getKey(), Long.toString(gen));
} else if (gen == SegmentInfo.NO) {
// No separate norm
} else {
// We should have already hit indexformat too old exception
assert false;
}
}
}
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile,
null, diagnostics, Collections.unmodifiableMap(attributes));
info.setFiles(files);
SegmentInfoPerCommit infoPerCommit = new SegmentInfoPerCommit(info, delCount, delGen);
return infoPerCommit;
}
private SegmentInfo readUpgradedSegmentInfo(String name, Directory dir, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene3xSegmentInfoFormat.UPGRADED_SI_CODEC_NAME,
Lucene3xSegmentInfoFormat.UPGRADED_SI_VERSION_START,
Lucene3xSegmentInfoFormat.UPGRADED_SI_VERSION_CURRENT);
final String version = input.readString();
final int docCount = input.readInt();
final Map<String,String> attributes = input.readStringStringMap();
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final Map<String,String> diagnostics = input.readStringStringMap();
final Set<String> files = input.readStringSet();
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile,
null, diagnostics, Collections.unmodifiableMap(attributes));
info.setFiles(files);
return info;
}
}

View File

@ -1,170 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.SegmentInfosReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Lucene 3x implementation of {@link SegmentInfosReader}.
* @lucene.experimental
* @deprecated
*/
@Deprecated
class Lucene3xSegmentInfosReader extends SegmentInfosReader {
@Override
public void read(Directory directory, String segmentsFileName, ChecksumIndexInput input, SegmentInfos infos, IOContext context) throws IOException {
infos.version = input.readLong(); // read version
infos.counter = input.readInt(); // read counter
final int format = infos.getFormat();
for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
SegmentInfo si = readSegmentInfo(directory, format, input);
if (si.getVersion() == null) {
// Could be a 3.0 - try to open the doc stores - if it fails, it's a
// 2.x segment, and an IndexFormatTooOldException will be thrown,
// which is what we want.
Directory dir = directory;
if (si.getDocStoreOffset() != -1) {
if (si.getDocStoreIsCompoundFile()) {
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
si.getDocStoreSegment(), "",
Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false);
}
} else if (si.getUseCompoundFile()) {
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
}
try {
Lucene3xStoredFieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
} finally {
// If we opened the directory, close it
if (dir != directory) dir.close();
}
// Above call succeeded, so it's a 3.0 segment. Upgrade it so the next
// time the segment is read, its version won't be null and we won't
// need to open FieldsReader every time for each such segment.
si.setVersion("3.0");
} else if (si.getVersion().equals("2.x")) {
// If it's a 3x index touched by 3.1+ code, then segments record their
// version, whether they are 2.x ones or not. We detect that and throw
// appropriate exception.
throw new IndexFormatTooOldException("segment " + si.name + " in resource " + input, si.getVersion());
}
infos.add(si);
}
infos.userData = input.readStringStringMap();
}
// if we make a preflex impl we can remove a lot of this hair...
public SegmentInfo readSegmentInfo(Directory dir, int format, ChecksumIndexInput input) throws IOException {
final String version;
if (format <= SegmentInfos.FORMAT_3_1) {
version = input.readString();
} else {
version = null;
}
final String name = input.readString();
final int docCount = input.readInt();
final long delGen = input.readLong();
final int docStoreOffset = input.readInt();
final String docStoreSegment;
final boolean docStoreIsCompoundFile;
if (docStoreOffset != -1) {
docStoreSegment = input.readString();
docStoreIsCompoundFile = input.readByte() == SegmentInfo.YES;
} else {
docStoreSegment = name;
docStoreIsCompoundFile = false;
}
// pre-4.0 indexes write a byte if there is a single norms file
byte b = input.readByte();
assert 1 == b : "expected 1 but was: "+ b + " format: " + format;
final int numNormGen = input.readInt();
final Map<Integer,Long> normGen;
if (numNormGen == SegmentInfo.NO) {
normGen = null;
} else {
normGen = new HashMap<Integer, Long>();
for(int j=0;j<numNormGen;j++) {
normGen.put(j, input.readLong());
}
}
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final int delCount = input.readInt();
assert delCount <= docCount;
final int hasProx = input.readByte();
final Codec codec = Codec.forName("Lucene3x");
final Map<String,String> diagnostics = input.readStringStringMap();
final int hasVectors;
if (format <= SegmentInfos.FORMAT_HAS_VECTORS) {
hasVectors = input.readByte();
} else {
final String storesSegment;
final String ext;
final boolean storeIsCompoundFile;
if (docStoreOffset != -1) {
storesSegment = docStoreSegment;
storeIsCompoundFile = docStoreIsCompoundFile;
ext = Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION;
} else {
storesSegment = name;
storeIsCompoundFile = isCompoundFile;
ext = IndexFileNames.COMPOUND_FILE_EXTENSION;
}
final Directory dirToTest;
if (storeIsCompoundFile) {
dirToTest = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE, false);
} else {
dirToTest = dir;
}
try {
hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION)) ? SegmentInfo.YES : SegmentInfo.NO;
} finally {
if (isCompoundFile) {
dirToTest.close();
}
}
}
return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
delCount, hasProx, codec, diagnostics, hasVectors);
}
}

View File

@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene3x;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
@ -39,13 +38,8 @@ class Lucene3xStoredFieldsFormat extends StoredFieldsFormat {
} }
@Override @Override
public StoredFieldsWriter fieldsWriter(Directory directory, String segment, public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si,
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

@ -36,7 +36,6 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import java.io.Closeable; import java.io.Closeable;
import java.util.Set;
/** /**
* Class responsible for access to stored document fields. * Class responsible for access to stored document fields.
@ -139,13 +138,13 @@ final class Lucene3xStoredFieldsReader extends StoredFieldsReader implements Clo
} }
public Lucene3xStoredFieldsReader(Directory d, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException { public Lucene3xStoredFieldsReader(Directory d, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
final String segment = si.getDocStoreSegment(); final String segment = Lucene3xSegmentInfoFormat.getDocStoreSegment(si);
final int docStoreOffset = si.getDocStoreOffset(); final int docStoreOffset = Lucene3xSegmentInfoFormat.getDocStoreOffset(si);
final int size = si.docCount; final int size = si.getDocCount();
boolean success = false; boolean success = false;
fieldInfos = fn; fieldInfos = fn;
try { try {
if (docStoreOffset != -1 && si.getDocStoreIsCompoundFile()) { if (docStoreOffset != -1 && Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(si)) {
d = storeCFSReader = new CompoundFileDirectory(si.dir, d = storeCFSReader = new CompoundFileDirectory(si.dir,
IndexFileNames.segmentFileName(segment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false); IndexFileNames.segmentFileName(segment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false);
} else { } else {
@ -176,8 +175,8 @@ final class Lucene3xStoredFieldsReader extends StoredFieldsReader implements Clo
this.docStoreOffset = 0; this.docStoreOffset = 0;
this.size = (int) (indexSize >> 3); this.size = (int) (indexSize >> 3);
// Verify two sources of "maxDoc" agree: // Verify two sources of "maxDoc" agree:
if (this.size != si.docCount) { if (this.size != si.getDocCount()) {
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.docCount); throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.getDocCount());
} }
} }
numTotalDocs = (int) (indexSize >> 3); numTotalDocs = (int) (indexSize >> 3);
@ -296,21 +295,4 @@ final class Lucene3xStoredFieldsReader extends StoredFieldsReader implements Clo
fieldsStream.seek(fieldsStream.getFilePointer() + length); fieldsStream.seek(fieldsStream.getFilePointer() + length);
} }
} }
// note: if there are shared docstores, we are also called by Lucene3xCodec even in
// the CFS case. so logic here must handle this.
public static void files(SegmentInfo info, Set<String> files) throws IOException {
if (info.getDocStoreOffset() != -1) {
assert info.getDocStoreSegment() != null;
if (info.getDocStoreIsCompoundFile()) {
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION));
} else {
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", FIELDS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", FIELDS_EXTENSION));
}
} else if (!info.getUseCompoundFile()) {
files.add(IndexFileNames.segmentFileName(info.name, "", FIELDS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(info.name, "", FIELDS_EXTENSION));
}
}
} }

View File

@ -18,13 +18,14 @@ package org.apache.lucene.codecs.lucene3x;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter; import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
@ -38,18 +39,42 @@ import org.apache.lucene.store.IOContext;
class Lucene3xTermVectorsFormat extends TermVectorsFormat { class Lucene3xTermVectorsFormat extends TermVectorsFormat {
@Override @Override
public TermVectorsReader vectorsReader(Directory directory,SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException { public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(Lucene3xSegmentInfoFormat.getDocStoreSegment(segmentInfo), "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION);
// Unfortunately, for 3.x indices, each segment's
// FieldInfos can lie about hasVectors (claim it's true
// when really it's false).... so we have to carefully
// check if the files really exist before trying to open
// them (4.x has fixed this):
final boolean exists;
if (Lucene3xSegmentInfoFormat.getDocStoreOffset(segmentInfo) != -1 && Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(segmentInfo)) {
String cfxFileName = IndexFileNames.segmentFileName(Lucene3xSegmentInfoFormat.getDocStoreSegment(segmentInfo), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION);
if (segmentInfo.dir.fileExists(cfxFileName)) {
Directory cfsDir = new CompoundFileDirectory(segmentInfo.dir, cfxFileName, context, false);
try {
exists = cfsDir.fileExists(fileName);
} finally {
cfsDir.close();
}
} else {
exists = false;
}
} else {
exists = directory.fileExists(fileName);
}
if (!exists) {
// 3x's FieldInfos sometimes lies and claims a segment
// has vectors when it doesn't:
return null;
} else {
return new Lucene3xTermVectorsReader(directory, segmentInfo, fieldInfos, context); return new Lucene3xTermVectorsReader(directory, segmentInfo, fieldInfos, context);
} }
}
@Override @Override
public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException { public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, 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

@ -22,7 +22,6 @@ import java.util.Arrays;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
@ -114,14 +113,14 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
public Lucene3xTermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context) public Lucene3xTermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context)
throws CorruptIndexException, IOException { throws CorruptIndexException, IOException {
final String segment = si.getDocStoreSegment(); final String segment = Lucene3xSegmentInfoFormat.getDocStoreSegment(si);
final int docStoreOffset = si.getDocStoreOffset(); final int docStoreOffset = Lucene3xSegmentInfoFormat.getDocStoreOffset(si);
final int size = si.docCount; final int size = si.getDocCount();
boolean success = false; boolean success = false;
try { try {
if (docStoreOffset != -1 && si.getDocStoreIsCompoundFile()) { if (docStoreOffset != -1 && Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(si)) {
d = storeCFSReader = new CompoundFileDirectory(si.dir, d = storeCFSReader = new CompoundFileDirectory(si.dir,
IndexFileNames.segmentFileName(segment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false); IndexFileNames.segmentFileName(segment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false);
} else { } else {
@ -239,7 +238,7 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
@Override @Override
public String next() throws IOException { public String next() throws IOException {
if (fieldNumbers != null && fieldUpto < fieldNumbers.length) { if (fieldNumbers != null && fieldUpto < fieldNumbers.length) {
return fieldInfos.fieldName(fieldNumbers[fieldUpto++]); return fieldInfos.fieldInfo(fieldNumbers[fieldUpto++]).name;
} else { } else {
return null; return null;
} }
@ -247,7 +246,7 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
@Override @Override
public Terms terms() throws IOException { public Terms terms() throws IOException {
return TVFields.this.terms(fieldInfos.fieldName(fieldNumbers[fieldUpto-1])); return TVFields.this.terms(fieldInfos.fieldInfo(fieldNumbers[fieldUpto-1]).name);
} }
}; };
} }
@ -690,27 +689,6 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
return new Lucene3xTermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, docStoreOffset, format); return new Lucene3xTermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, docStoreOffset, format);
} }
// note: if there are shared docstores, we are also called by Lucene3xCodec even in
// the CFS case. so logic here must handle this.
public static void files(SegmentInfo info, Set<String> files) throws IOException {
if (info.getHasVectors()) {
if (info.getDocStoreOffset() != -1) {
assert info.getDocStoreSegment() != null;
if (info.getDocStoreIsCompoundFile()) {
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION));
} else {
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_FIELDS_EXTENSION));
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_DOCUMENTS_EXTENSION));
}
} else if (!info.getUseCompoundFile()) {
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
}
}
}
// 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
// true when segments are used for "normal" searching; // true when segments are used for "normal" searching;

View File

@ -89,8 +89,8 @@ class SegmentTermDocs {
void seek(TermInfo ti, Term term) throws IOException { void seek(TermInfo ti, Term term) throws IOException {
count = 0; count = 0;
FieldInfo fi = fieldInfos.fieldInfo(term.field()); FieldInfo fi = fieldInfos.fieldInfo(term.field());
this.indexOptions = (fi != null) ? fi.indexOptions : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS; this.indexOptions = (fi != null) ? fi.getIndexOptions() : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
currentFieldStoresPayloads = (fi != null) ? fi.storePayloads : false; currentFieldStoresPayloads = (fi != null) ? fi.hasPayloads() : false;
if (ti == null) { if (ti == null) {
df = 0; df = 0;
} else { } else {

View File

@ -67,9 +67,15 @@ final class TermBuffer implements Cloneable {
final int fieldNumber = input.readVInt(); final int fieldNumber = input.readVInt();
if (fieldNumber != currentFieldNumber) { if (fieldNumber != currentFieldNumber) {
currentFieldNumber = fieldNumber; currentFieldNumber = fieldNumber;
field = fieldInfos.fieldName(currentFieldNumber).intern(); // NOTE: too much sneakiness here, seriously this is a negative vint?!
if (currentFieldNumber == -1) {
field = "";
} else { } else {
assert field.equals(fieldInfos.fieldName(fieldNumber)): "currentFieldNumber=" + currentFieldNumber + " field=" + field + " vs " + fieldInfos.fieldName(fieldNumber); assert fieldInfos.fieldInfo(currentFieldNumber) != null : currentFieldNumber;
field = fieldInfos.fieldInfo(currentFieldNumber).name.intern();
}
} else {
assert field.equals(fieldInfos.fieldInfo(fieldNumber).name) : "currentFieldNumber=" + currentFieldNumber + " field=" + field + " vs " + fieldInfos.fieldInfo(fieldNumber) == null ? "null" : fieldInfos.fieldInfo(fieldNumber).name;
} }
} }

View File

@ -23,7 +23,7 @@ import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -42,7 +42,7 @@ public class Lucene40Codec extends Codec {
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat(); private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat(); private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat(); private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfosFormat infosFormat = new Lucene40SegmentInfosFormat(); private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat(); private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat(); private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
@ -83,7 +83,7 @@ public class Lucene40Codec extends Codec {
} }
@Override @Override
public SegmentInfosFormat segmentInfosFormat() { public SegmentInfoFormat segmentInfoFormat() {
return infosFormat; return infosFormat;
} }

View File

@ -18,14 +18,10 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase; import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.CompoundFileDirectory; import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
@ -68,24 +64,12 @@ public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
} }
} }
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 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;
}
}
}
@Override @Override
public void abort() { public void abort() {
try { try {
close(); close();
} catch (IOException ignored) {} } catch (IOException ignored) {
}
IOUtils.deleteFilesIgnoringExceptions(mainDirectory, IndexFileNames.segmentFileName( IOUtils.deleteFilesIgnoringExceptions(mainDirectory, IndexFileNames.segmentFileName(
segmentName, segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION), segmentName, segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
IndexFileNames.segmentFileName(segmentName, segmentSuffix, IndexFileNames.segmentFileName(segmentName, segmentSuffix,

View File

@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PerDocConsumer; import org.apache.lucene.codecs.PerDocConsumer;
@ -26,7 +25,6 @@ import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.DocValues; // javadocs import org.apache.lucene.index.DocValues; // javadocs
import org.apache.lucene.index.DocValues.Type; // javadocs import org.apache.lucene.index.DocValues.Type; // javadocs
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.CompoundFileDirectory; // javadocs import org.apache.lucene.store.CompoundFileDirectory; // javadocs
import org.apache.lucene.store.DataOutput; // javadocs import org.apache.lucene.store.DataOutput; // javadocs
@ -141,9 +139,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

@ -56,7 +56,7 @@ public class Lucene40DocValuesProducer extends PerDocProducerBase {
IndexFileNames.segmentFileName(state.segmentInfo.name, IndexFileNames.segmentFileName(state.segmentInfo.name,
segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION), segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
state.context, false); state.context, false);
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, cfs, state.context); docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.getDocCount(), cfs, state.context);
} else { } else {
cfs = null; cfs = null;
docValues = new TreeMap<String,DocValues>(); docValues = new TreeMap<String,DocValues>();

View File

@ -18,33 +18,33 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosReader; import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter; import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.DocValues; // javadoc import org.apache.lucene.index.DocValues; // javadoc
import org.apache.lucene.index.DocValues.Type; // javadoc import org.apache.lucene.index.DocValues.Type; // javadoc
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.DataOutput; // javadoc import org.apache.lucene.store.DataOutput; // javadoc
import org.apache.lucene.util.CodecUtil; // javadoc
/** /**
* Lucene 4.0 Field Infos format. * Lucene 4.0 Field Infos format.
* <p> * <p>
* <p>Field names are stored in the field info file, with suffix <tt>.fnm</tt>.</p> * <p>Field names are stored in the field info file, with suffix <tt>.fnm</tt>.</p>
* <p>FieldInfos (.fnm) --&gt; FNMVersion,FieldsCount, &lt;FieldName,FieldNumber, * <p>FieldInfos (.fnm) --&gt; Header,FieldsCount, &lt;FieldName,FieldNumber,
* FieldBits,DocValuesBits&gt; <sup>FieldsCount</sup></p> * FieldBits,DocValuesBits,Attributes&gt; <sup>FieldsCount</sup></p>
* <p>Data types: * <p>Data types:
* <ul> * <ul>
* <li>FNMVersion, FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li> * <li>Header --&gt; {@link CodecUtil#checkHeader CodecHeader}</li>
* <li>FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>FieldName --&gt; {@link DataOutput#writeString String}</li> * <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
* <li>FieldBits, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li> * <li>FieldBits, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>FieldNumber --&gt; {@link DataOutput#writeInt Uint32}</li> * <li>FieldNumber --&gt; {@link DataOutput#writeInt VInt}</li>
* <li>Attributes --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
* </ul> * </ul>
* </p> * </p>
* Field Descriptions: * Field Descriptions:
* <ul> * <ul>
* <li>FNMVersion is <code>Lucene40FieldInfosWriter.FORMAT_CURRENT</code>.</li>
* <li>FieldsCount: the number of fields in this file.</li> * <li>FieldsCount: the number of fields in this file.</li>
* <li>FieldName: name of the field as a UTF-8 String.</li> * <li>FieldName: name of the field as a UTF-8 String.</li>
* <li>FieldNumber: the field's number. Note that unlike previous versions of * <li>FieldNumber: the field's number. Note that unlike previous versions of
@ -90,6 +90,7 @@ import org.apache.lucene.store.DataOutput; // javadoc
* <li>13: variable-length sorted byte array values. ({@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED})</li> * <li>13: variable-length sorted byte array values. ({@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED})</li>
* </ul> * </ul>
* </li> * </li>
* <li>Attributes: a key-value map of codec-private attributes.</li>
* </ul> * </ul>
* *
* @lucene.experimental * @lucene.experimental
@ -107,9 +108,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

@ -1,21 +1,20 @@
package org.apache.lucene.codecs.lucene40; package org.apache.lucene.codecs.lucene40;
import java.io.IOException; import java.io.IOException;
import java.util.Set; import java.util.Collections;
import java.util.Map;
import org.apache.lucene.codecs.FieldInfosReader; import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValues;
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.IndexInput; import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
@ -42,33 +41,22 @@ import org.apache.lucene.store.IndexInput;
*/ */
public class Lucene40FieldInfosReader extends FieldInfosReader { public class Lucene40FieldInfosReader extends FieldInfosReader {
static final int FORMAT_MINIMUM = Lucene40FieldInfosWriter.FORMAT_START;
@Override @Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException { public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosWriter.FIELD_INFOS_EXTENSION); final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosWriter.FIELD_INFOS_EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext); IndexInput input = directory.openInput(fileName, iocontext);
boolean hasVectors = false;
boolean hasFreq = false;
boolean hasProx = false;
try { try {
final int format = input.readVInt(); CodecUtil.checkHeader(input, Lucene40FieldInfosWriter.CODEC_NAME,
Lucene40FieldInfosWriter.FORMAT_START,
if (format > FORMAT_MINIMUM) { Lucene40FieldInfosWriter.FORMAT_CURRENT);
throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, Lucene40FieldInfosWriter.FORMAT_CURRENT);
}
if (format < Lucene40FieldInfosWriter.FORMAT_CURRENT) {
throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, Lucene40FieldInfosWriter.FORMAT_CURRENT);
}
final int size = input.readVInt(); //read in the size final int size = input.readVInt(); //read in the size
FieldInfo infos[] = new FieldInfo[size]; FieldInfo infos[] = new FieldInfo[size];
for (int i = 0; i < size; i++) { for (int i = 0; i < size; i++) {
String name = input.readString(); String name = input.readString();
final int fieldNumber = input.readInt(); final int fieldNumber = input.readVInt();
byte bits = input.readByte(); byte bits = input.readByte();
boolean isIndexed = (bits & Lucene40FieldInfosWriter.IS_INDEXED) != 0; boolean isIndexed = (bits & Lucene40FieldInfosWriter.IS_INDEXED) != 0;
boolean storeTermVector = (bits & Lucene40FieldInfosWriter.STORE_TERMVECTOR) != 0; boolean storeTermVector = (bits & Lucene40FieldInfosWriter.STORE_TERMVECTOR) != 0;
@ -91,22 +79,20 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
storePayloads = false; storePayloads = false;
} }
hasVectors |= storeTermVector;
hasProx |= isIndexed && indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
// DV Types are packed in one byte // DV Types are packed in one byte
byte val = input.readByte(); byte val = input.readByte();
final DocValues.Type docValuesType = getDocValuesType((byte) (val & 0x0F)); final DocValues.Type docValuesType = getDocValuesType((byte) (val & 0x0F));
final DocValues.Type normsType = getDocValuesType((byte) ((val >>> 4) & 0x0F)); final DocValues.Type normsType = getDocValuesType((byte) ((val >>> 4) & 0x0F));
final Map<String,String> attributes = input.readStringStringMap();
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, docValuesType, normsType); omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes));
} }
if (input.getFilePointer() != input.length()) { if (input.getFilePointer() != input.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
} }
return new FieldInfos(infos, hasFreq, hasProx, hasVectors); return new FieldInfos(infos);
} finally { } finally {
input.close(); input.close();
} }
@ -147,8 +133,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

@ -27,6 +27,7 @@ import org.apache.lucene.index.FieldInfo.IndexOptions;
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.IndexOutput; import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.CodecUtil;
/** /**
* Lucene 4.0 FieldInfos writer. * Lucene 4.0 FieldInfos writer.
@ -39,10 +40,8 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
/** Extension of field infos */ /** Extension of field infos */
static final String FIELD_INFOS_EXTENSION = "fnm"; static final String FIELD_INFOS_EXTENSION = "fnm";
// per-field codec support, records index values for fields static final String CODEC_NAME = "Lucene40FieldInfos";
static final int FORMAT_START = -4; static final int FORMAT_START = 0;
// whenever you add a new format, make it 1 smaller (negative version logic)!
static final int FORMAT_CURRENT = FORMAT_START; static final int FORMAT_CURRENT = FORMAT_START;
static final byte IS_INDEXED = 0x1; static final byte IS_INDEXED = 0x1;
@ -58,24 +57,25 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION); final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
IndexOutput output = directory.createOutput(fileName, context); IndexOutput output = directory.createOutput(fileName, context);
try { try {
output.writeVInt(FORMAT_CURRENT); CodecUtil.writeHeader(output, CODEC_NAME, FORMAT_CURRENT);
output.writeVInt(infos.size()); output.writeVInt(infos.size());
for (FieldInfo fi : infos) { for (FieldInfo fi : infos) {
assert fi.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.storePayloads; IndexOptions indexOptions = fi.getIndexOptions();
assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
byte bits = 0x0; byte bits = 0x0;
if (fi.isIndexed) bits |= IS_INDEXED; if (fi.isIndexed()) bits |= IS_INDEXED;
if (fi.storeTermVector) bits |= STORE_TERMVECTOR; if (fi.hasVectors()) bits |= STORE_TERMVECTOR;
if (fi.omitNorms) bits |= OMIT_NORMS; if (fi.omitsNorms()) bits |= OMIT_NORMS;
if (fi.storePayloads) bits |= STORE_PAYLOADS; if (fi.hasPayloads()) bits |= STORE_PAYLOADS;
if (fi.indexOptions == IndexOptions.DOCS_ONLY) { if (indexOptions == IndexOptions.DOCS_ONLY) {
bits |= OMIT_TERM_FREQ_AND_POSITIONS; bits |= OMIT_TERM_FREQ_AND_POSITIONS;
} else if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) { } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
bits |= STORE_OFFSETS_IN_POSTINGS; bits |= STORE_OFFSETS_IN_POSTINGS;
} else if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS) { } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
bits |= OMIT_POSITIONS; bits |= OMIT_POSITIONS;
} }
output.writeString(fi.name); output.writeString(fi.name);
output.writeInt(fi.number); output.writeVInt(fi.number);
output.writeByte(bits); output.writeByte(bits);
// pack the DV types in one byte // pack the DV types in one byte
@ -84,6 +84,7 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
assert (dv & (~0xF)) == 0 && (nrm & (~0x0F)) == 0; assert (dv & (~0xF)) == 0 && (nrm & (~0x0F)) == 0;
byte val = (byte) (0xff & ((nrm << 4) | dv)); byte val = (byte) (0xff & ((nrm << 4) | dv));
output.writeByte(val); output.writeByte(val);
output.writeStringStringMap(fi.attributes());
} }
} finally { } finally {
output.close(); output.close();

View File

@ -18,11 +18,11 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set; import java.util.Collection;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfoPerCommit;
import org.apache.lucene.store.DataOutput; // javadocs import org.apache.lucene.store.DataOutput; // javadocs
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
@ -81,27 +81,28 @@ public class Lucene40LiveDocsFormat extends LiveDocsFormat {
} }
@Override @Override
public Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) throws IOException { public Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) throws IOException {
String filename = IndexFileNames.fileNameFromGeneration(info.name, DELETES_EXTENSION, info.getDelGen()); String filename = IndexFileNames.fileNameFromGeneration(info.info.name, DELETES_EXTENSION, info.getDelGen());
final BitVector liveDocs = new BitVector(dir, filename, context); final BitVector liveDocs = new BitVector(dir, filename, context);
assert liveDocs.count() == info.docCount - info.getDelCount(); assert liveDocs.count() == info.info.getDocCount() - info.getDelCount():
assert liveDocs.length() == info.docCount; "liveDocs.count()=" + liveDocs.count() + " info.docCount=" + info.info.getDocCount() + " info.getDelCount()=" + info.getDelCount();
assert liveDocs.length() == info.info.getDocCount();
return liveDocs; return liveDocs;
} }
@Override @Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException { public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, IOContext context) throws IOException {
String filename = IndexFileNames.fileNameFromGeneration(info.name, DELETES_EXTENSION, info.getDelGen()); String filename = IndexFileNames.fileNameFromGeneration(info.info.name, DELETES_EXTENSION, info.getNextDelGen());
final BitVector liveDocs = (BitVector) bits; final BitVector liveDocs = (BitVector) bits;
assert liveDocs.count() == info.docCount - info.getDelCount(); assert liveDocs.count() == info.info.getDocCount() - info.getDelCount() - newDelCount;
assert liveDocs.length() == info.docCount; assert liveDocs.length() == info.info.getDocCount();
liveDocs.write(dir, filename, context); liveDocs.write(dir, filename, context);
} }
@Override @Override
public void files(SegmentInfo info, Set<String> files) throws IOException { public void files(SegmentInfoPerCommit info, Collection<String> files) throws IOException {
if (info.hasDeletions()) { if (info.hasDeletions()) {
files.add(IndexFileNames.fileNameFromGeneration(info.name, DELETES_EXTENSION, info.getDelGen())); files.add(IndexFileNames.fileNameFromGeneration(info.info.name, DELETES_EXTENSION, info.getDelGen()));
} }
} }
} }

View File

@ -1,4 +1,5 @@
package org.apache.lucene.codecs.lucene40; package org.apache.lucene.codecs.lucene40;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
@ -15,8 +16,8 @@ package org.apache.lucene.codecs.lucene40;
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer; import org.apache.lucene.codecs.PerDocConsumer;
@ -25,10 +26,8 @@ import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type; import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.AtomicReader; import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.CompoundFileDirectory; // javadocs import org.apache.lucene.store.CompoundFileDirectory; // javadocs
@ -59,11 +58,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.
* *
@ -121,19 +115,5 @@ public class Lucene40NormsFormat extends NormsFormat {
protected Type getDocValuesType(FieldInfo info) { protected Type getDocValuesType(FieldInfo info) {
return info.getNormType(); return info.getNormType();
} }
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) {
if (fieldInfo.hasNorms()) {
final String normsEntriesFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
files.add(normsFileName);
files.add(normsEntriesFileName);
return;
} }
}
}
}
} }

View File

@ -18,12 +18,10 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.PostingsBaseFormat; import org.apache.lucene.codecs.PostingsBaseFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -42,16 +40,11 @@ public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
@Override @Override
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException { public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
return new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); return new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
} }
@Override @Override
public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException { public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
return new Lucene40PostingsWriter(state); return new Lucene40PostingsWriter(state);
} }
@Override
public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
Lucene40PostingsReader.files(segmentInfo, segmentSuffix, files);
}
} }

View File

@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.BlockTreeTermsReader; import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter; import org.apache.lucene.codecs.BlockTreeTermsWriter;
@ -30,7 +29,6 @@ import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.DocsEnum; // javadocs import org.apache.lucene.index.DocsEnum; // javadocs
import org.apache.lucene.index.FieldInfo.IndexOptions; // javadocs import org.apache.lucene.index.FieldInfo.IndexOptions; // javadocs
import org.apache.lucene.index.FieldInfos; // javadocs import org.apache.lucene.index.FieldInfos; // javadocs
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.DataOutput; // javadocs import org.apache.lucene.store.DataOutput; // javadocs
@ -307,7 +305,7 @@ public class Lucene40PostingsFormat extends PostingsFormat {
@Override @Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
boolean success = false; boolean success = false;
try { try {
@ -334,12 +332,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

@ -18,14 +18,14 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState; import org.apache.lucene.index.TermState;
@ -57,11 +57,18 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
// private String segment; // private String segment;
public Lucene40PostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException { public Lucene40PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION), freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
ioContext); ioContext);
// this.segment = segmentInfo.name; // TODO: hasProx should (somehow!) become codec private,
if (segmentInfo.getHasProx()) { // but it's tricky because 1) FIS.hasProx is global (it
// could be all fields that have prox are written by a
// different codec), 2) the field may have had prox in
// the past but all docs w/ that field were deleted.
// Really we'd need to init prxOut lazily on write, and
// then somewhere record that we actually wrote it so we
// know whether to open on read:
if (fieldInfos.hasProx()) {
boolean success = false; boolean success = false;
try { try {
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION), proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION),
@ -77,13 +84,6 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
} }
} }
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));
}
}
@Override @Override
public void init(IndexInput termsIn) throws IOException { public void init(IndexInput termsIn) throws IOException {
@ -200,7 +200,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
// undefined // undefined
} }
if (fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) { if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
if (isFirstTerm) { if (isFirstTerm) {
termState.proxOffset = termState.bytesReader.readVLong(); termState.proxOffset = termState.bytesReader.readVLong();
} else { } else {
@ -212,7 +212,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
@Override @Override
public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException { public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
if (needsFreqs && fieldInfo.indexOptions == IndexOptions.DOCS_ONLY) { if (needsFreqs && fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY) {
return null; return null;
} else if (canReuse(reuse, liveDocs)) { } else if (canReuse(reuse, liveDocs)) {
// if (DEBUG) System.out.println("SPR.docs ts=" + termState); // if (DEBUG) System.out.println("SPR.docs ts=" + termState);
@ -248,13 +248,13 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
DocsAndPositionsEnum reuse, boolean needsOffsets) DocsAndPositionsEnum reuse, boolean needsOffsets)
throws IOException { throws IOException {
boolean hasOffsets = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; boolean hasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (needsOffsets && !hasOffsets) { if (needsOffsets && !hasOffsets) {
return null; // not available return null; // not available
} }
// TODO: refactor // TODO: refactor
if (fieldInfo.storePayloads || hasOffsets) { if (fieldInfo.hasPayloads() || hasOffsets) {
SegmentFullPositionsEnum docsEnum; SegmentFullPositionsEnum docsEnum;
if (reuse == null || !(reuse instanceof SegmentFullPositionsEnum)) { if (reuse == null || !(reuse instanceof SegmentFullPositionsEnum)) {
docsEnum = new SegmentFullPositionsEnum(freqIn, proxIn); docsEnum = new SegmentFullPositionsEnum(freqIn, proxIn);
@ -326,9 +326,9 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
DocsEnum reset(FieldInfo fieldInfo, StandardTermState termState) throws IOException { DocsEnum reset(FieldInfo fieldInfo, StandardTermState termState) throws IOException {
indexOmitsTF = fieldInfo.indexOptions == IndexOptions.DOCS_ONLY; indexOmitsTF = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY;
storePayloads = fieldInfo.storePayloads; storePayloads = fieldInfo.hasPayloads();
storeOffsets = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; storeOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
freqOffset = termState.freqOffset; freqOffset = termState.freqOffset;
skipOffset = termState.skipOffset; skipOffset = termState.skipOffset;
@ -701,8 +701,8 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
} }
public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException { public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
assert fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS; assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
assert !fieldInfo.storePayloads; assert !fieldInfo.hasPayloads();
this.liveDocs = liveDocs; this.liveDocs = liveDocs;
@ -914,9 +914,9 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
} }
public SegmentFullPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException { public SegmentFullPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
storeOffsets = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; storeOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
storePayloads = fieldInfo.storePayloads; storePayloads = fieldInfo.hasPayloads();
assert fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; assert fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
assert storePayloads || storeOffsets; assert storePayloads || storeOffsets;
if (payload == null) { if (payload == null) {
payload = new BytesRef(); payload = new BytesRef();

View File

@ -99,14 +99,18 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
this.skipInterval = skipInterval; this.skipInterval = skipInterval;
this.skipMinimum = skipInterval; /* set to the same for now */ this.skipMinimum = skipInterval; /* set to the same for now */
// this.segment = state.segmentName; // this.segment = state.segmentName;
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION); String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION);
freqOut = state.directory.createOutput(fileName, state.context); freqOut = state.directory.createOutput(fileName, state.context);
boolean success = false; boolean success = false;
try { try {
// TODO: this is a best effort, if one of these fields has no postings
// then we make an empty prx file, same as if we are wrapped in
// per-field postingsformat. maybe... we shouldn't
// bother w/ this opto? just create empty prx file...?
if (state.fieldInfos.hasProx()) { if (state.fieldInfos.hasProx()) {
// At least one field does not omit TF, so create the // At least one field does not omit TF, so create the
// prox file // prox file
fileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION); fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION);
proxOut = state.directory.createOutput(fileName, state.context); proxOut = state.directory.createOutput(fileName, state.context);
} else { } else {
// Every field omits TF so we will write no prox file // Every field omits TF so we will write no prox file
@ -119,11 +123,11 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
} }
} }
totalNumDocs = state.numDocs; totalNumDocs = state.segmentInfo.getDocCount();
skipListWriter = new Lucene40SkipListWriter(skipInterval, skipListWriter = new Lucene40SkipListWriter(skipInterval,
maxSkipLevels, maxSkipLevels,
state.numDocs, totalNumDocs,
freqOut, freqOut,
proxOut); proxOut);
} }
@ -164,10 +168,10 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
} }
*/ */
this.fieldInfo = fieldInfo; this.fieldInfo = fieldInfo;
indexOptions = fieldInfo.indexOptions; indexOptions = fieldInfo.getIndexOptions();
storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
storePayloads = fieldInfo.storePayloads; storePayloads = fieldInfo.hasPayloads();
//System.out.println(" set init blockFreqStart=" + freqStart); //System.out.println(" set init blockFreqStart=" + freqStart);
//System.out.println(" set init blockProxStart=" + proxStart); //System.out.println(" set init blockProxStart=" + proxStart);
} }

View File

@ -0,0 +1,88 @@
package org.apache.lucene.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.codecs.SegmentInfoWriter;
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.SegmentInfos; // javadocs
import org.apache.lucene.store.DataOutput; // javadocs
import org.apache.lucene.util.CodecUtil; // javadocs
/**
* Lucene 4.0 Segment info format.
* <p>
* Files:
* <ul>
* <li><tt>.si</tt>: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Attributes, Files
* </ul>
* </p>
* Data types:
* <p>
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
* <li>Files --&gt; {@link DataOutput#writeStringSet Set&lt;String&gt;}</li>
* <li>Diagnostics, Attributes --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
* <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
* </ul>
* </p>
* Field Descriptions:
* <p>
* <ul>
* <li>SegVersion is the code version that created the segment.</li>
* <li>SegSize is the number of documents contained in the segment index.</li>
* <li>IsCompoundFile records whether the segment is written as a compound file or
* not. If this is -1, the segment is not a compound file. If it is 1, the segment
* is a compound file.</li>
* <li>Checksum contains the CRC32 checksum of all bytes in the segments_N file up
* until the checksum. This is used to verify integrity of the file on opening the
* index.</li>
* <li>The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid,
* for each segment it creates. It includes metadata like the current Lucene
* version, OS, Java version, why the segment was created (merge, flush,
* addIndexes), etc.</li>
* <li>Attributes: a key-value map of codec-private attributes.</li>
* <li>Files is a list of files referred to by this segment.</li>
* </ul>
* </p>
*
* @see SegmentInfos
* @lucene.experimental
*/
public class Lucene40SegmentInfoFormat extends SegmentInfoFormat {
private final SegmentInfoReader reader = new Lucene40SegmentInfoReader();
private final SegmentInfoWriter writer = new Lucene40SegmentInfoWriter();
@Override
public SegmentInfoReader getSegmentInfosReader() {
return reader;
}
@Override
public SegmentInfoWriter getSegmentInfosWriter() {
return writer;
}
public final static String SI_EXTENSION = "si";
static final String CODEC_NAME = "Lucene40SegmentInfo";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
}

View File

@ -0,0 +1,74 @@
package org.apache.lucene.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.0 implementation of {@link SegmentInfoReader}.
*
* @see Lucene40SegmentInfoFormat
* @lucene.experimental
*/
public class Lucene40SegmentInfoReader extends SegmentInfoReader {
@Override
public SegmentInfo read(Directory dir, String segment, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene40SegmentInfoFormat.SI_EXTENSION);
final IndexInput input = dir.openInput(fileName, context);
boolean success = false;
try {
CodecUtil.checkHeader(input, Lucene40SegmentInfoFormat.CODEC_NAME,
Lucene40SegmentInfoFormat.VERSION_START,
Lucene40SegmentInfoFormat.VERSION_CURRENT);
final String version = input.readString();
final int docCount = input.readInt();
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final Map<String,String> diagnostics = input.readStringStringMap();
final Map<String,String> attributes = input.readStringStringMap();
final Set<String> files = input.readStringSet();
final SegmentInfo si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile,
null, diagnostics, Collections.unmodifiableMap(attributes));
si.setFiles(files);
success = true;
return si;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(input);
} else {
input.close();
}
}
}
}

View File

@ -0,0 +1,70 @@
package org.apache.lucene.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.SegmentInfoWriter;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.0 implementation of {@link SegmentInfoWriter}.
*
* @see Lucene40SegmentInfoFormat
* @lucene.experimental
*/
public class Lucene40SegmentInfoWriter extends SegmentInfoWriter {
/** Save a single segment's info. */
@Override
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene40SegmentInfoFormat.SI_EXTENSION);
si.addFile(fileName);
final IndexOutput output = dir.createOutput(fileName, ioContext);
boolean success = false;
try {
CodecUtil.writeHeader(output, Lucene40SegmentInfoFormat.CODEC_NAME, Lucene40SegmentInfoFormat.VERSION_CURRENT);
// Write the Lucene version that created this segment, since 3.1
output.writeString(si.getVersion());
output.writeInt(si.getDocCount());
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
output.writeStringStringMap(si.getDiagnostics());
output.writeStringStringMap(si.attributes());
output.writeStringSet(si.files());
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
si.dir.deleteFile(fileName);
} else {
output.close();
}
}
}
}

View File

@ -1,132 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.Codec; // javadocs
import org.apache.lucene.codecs.LiveDocsFormat; // javadocs
import org.apache.lucene.codecs.SegmentInfosFormat;
import org.apache.lucene.codecs.SegmentInfosReader;
import org.apache.lucene.codecs.SegmentInfosWriter;
import org.apache.lucene.codecs.StoredFieldsFormat; // javadocs
import org.apache.lucene.codecs.TermVectorsFormat; // javadocs
import org.apache.lucene.index.FieldInfo.IndexOptions; // javadocs
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.SegmentInfos; // javadocs
import org.apache.lucene.store.DataOutput; // javadocs
/**
* Lucene 4.0 Segments format.
* <p>
* Files:
* <ul>
* <li><tt>segments.gen</tt>: described in {@link SegmentInfos}
* <li><tt>segments_N</tt>: Format, Codec, Version, NameCounter, SegCount,
* &lt;SegVersion, SegName, SegSize, DelGen, DocStoreOffset, [DocStoreSegment,
* DocStoreIsCompoundFile], NumField, NormGen<sup>NumField</sup>,
* IsCompoundFile, DeletionCount, HasProx, SegCodec Diagnostics,
* HasVectors&gt;<sup>SegCount</sup>, CommitUserData, Checksum
* </ul>
* </p>
* Data types:
* <p>
* <ul>
* <li>Format, NameCounter, SegCount, SegSize, NumField, DocStoreOffset,
* DeletionCount --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>Version, DelGen, NormGen, Checksum --&gt;
* {@link DataOutput#writeLong Int64}</li>
* <li>SegVersion, SegName, DocStoreSegment, Codec, SegCodec --&gt;
* {@link DataOutput#writeString String}</li>
* <li>Diagnostics, CommitUserData --&gt;
* {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
* <li>IsCompoundFile, DocStoreIsCompoundFile, HasProx,
* HasVectors --&gt; {@link DataOutput#writeByte Int8}</li>
* </ul>
* </p>
* Field Descriptions:
* <p>
* <ul>
* <li>Format is {@link SegmentInfos#FORMAT_4_0}.</li>
* <li>Codec is "Lucene40", its the {@link Codec} that wrote this particular segments file.</li>
* <li>Version counts how often the index has been changed by adding or deleting
* documents.</li>
* <li>NameCounter is used to generate names for new segment files.</li>
* <li>SegVersion is the code version that created the segment.</li>
* <li>SegName is the name of the segment, and is used as the file name prefix for
* all of the files that compose the segment's index.</li>
* <li>SegSize is the number of documents contained in the segment index.</li>
* <li>DelGen is the generation count of the deletes file. If this is -1,
* there are no deletes. Anything above zero means there are deletes
* stored by {@link LiveDocsFormat}.</li>
* <li>NumField is the size of the array for NormGen, or -1 if there are no
* NormGens stored.</li>
* <li>NormGen records the generation of the separate norms files. If NumField is
* -1, there are no normGens stored and all assumed to be -1. The generation
* then has the same meaning as delGen (above).</li>
* <li>IsCompoundFile records whether the segment is written as a compound file or
* not. If this is -1, the segment is not a compound file. If it is 1, the segment
* is a compound file. Else it is 0, which means we check filesystem to see if
* _X.cfs exists.</li>
* <li>DocStoreOffset, DocStoreSegment, DocStoreIsCompoundFile: If DocStoreOffset
* is -1, this segment has its own doc store (stored fields values and term
* vectors) files and DocStoreSegment and DocStoreIsCompoundFile are not stored.
* In this case all files for {@link StoredFieldsFormat stored field values} and
* {@link TermVectorsFormat term vectors} will be stored with this segment.
* Otherwise, DocStoreSegment is the name of the segment that has the shared doc
* store files; DocStoreIsCompoundFile is 1 if that segment is stored in compound
* file format (as a <tt>.cfx</tt> file); and DocStoreOffset is the starting document
* in the shared doc store files where this segment's documents begin. In this case,
* this segment does not store its own doc store files but instead shares a single
* set of these files with other segments.</li>
* <li>Checksum contains the CRC32 checksum of all bytes in the segments_N file up
* until the checksum. This is used to verify integrity of the file on opening the
* index.</li>
* <li>DeletionCount records the number of deleted documents in this segment.</li>
* <li>HasProx is 1 if any fields in this segment have position data
* ({@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS DOCS_AND_FREQS_AND_POSITIONS} or
* {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS});
* else, it's 0.</li>
* <li>SegCodec is the {@link Codec#getName() name} of the Codec that encoded
* this segment.</li>
* <li>CommitUserData stores an optional user-supplied opaque
* Map&lt;String,String&gt; that was passed to {@link IndexWriter#commit(java.util.Map)}
* or {@link IndexWriter#prepareCommit(java.util.Map)}.</li>
* <li>The Diagnostics Map is privately written by IndexWriter, as a debugging aid,
* for each segment it creates. It includes metadata like the current Lucene
* version, OS, Java version, why the segment was created (merge, flush,
* addIndexes), etc.</li>
* <li>HasVectors is 1 if this segment stores term vectors, else it's 0.</li>
* </ul>
* </p>
*
* @see SegmentInfos
* @lucene.experimental
*/
public class Lucene40SegmentInfosFormat extends SegmentInfosFormat {
private final SegmentInfosReader reader = new Lucene40SegmentInfosReader();
private final SegmentInfosWriter writer = new Lucene40SegmentInfosWriter();
@Override
public SegmentInfosReader getSegmentInfosReader() {
return reader;
}
@Override
public SegmentInfosWriter getSegmentInfosWriter() {
return writer;
}
}

View File

@ -1,94 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.SegmentInfosReader;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Lucene 4.0 implementation of {@link SegmentInfosReader}.
*
* @see Lucene40SegmentInfosFormat
* @lucene.experimental
*/
public class Lucene40SegmentInfosReader extends SegmentInfosReader {
@Override
public void read(Directory directory, String segmentsFileName, ChecksumIndexInput input, SegmentInfos infos, IOContext context) throws IOException {
infos.version = input.readLong(); // read version
infos.counter = input.readInt(); // read counter
final int format = infos.getFormat();
assert format <= SegmentInfos.FORMAT_4_0;
for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
SegmentInfo si = readSegmentInfo(directory, format, input);
assert si.getVersion() != null;
infos.add(si);
}
infos.userData = input.readStringStringMap();
}
public SegmentInfo readSegmentInfo(Directory dir, int format, ChecksumIndexInput input) throws IOException {
final String version = input.readString();
final String name = input.readString();
final int docCount = input.readInt();
final long delGen = input.readLong();
// this is still written in 4.0 if we open a 3.x and upgrade the SI
final int docStoreOffset = input.readInt();
final String docStoreSegment;
final boolean docStoreIsCompoundFile;
if (docStoreOffset != -1) {
docStoreSegment = input.readString();
docStoreIsCompoundFile = input.readByte() == SegmentInfo.YES;
} else {
docStoreSegment = name;
docStoreIsCompoundFile = false;
}
final int numNormGen = input.readInt();
final Map<Integer,Long> normGen;
if (numNormGen == SegmentInfo.NO) {
normGen = null;
} else {
normGen = new HashMap<Integer, Long>();
for(int j=0;j<numNormGen;j++) {
normGen.put(input.readInt(), input.readLong());
}
}
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final int delCount = input.readInt();
assert delCount <= docCount;
final int hasProx = input.readByte();
final Codec codec = Codec.forName(input.readString());
final Map<String,String> diagnostics = input.readStringStringMap();
final int hasVectors = input.readByte();
return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
delCount, hasProx, codec, diagnostics, hasVectors);
}
}

View File

@ -1,121 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.lucene.codecs.SegmentInfosWriter;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.ChecksumIndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.0 implementation of {@link SegmentInfosWriter}.
*
* @see Lucene40SegmentInfosFormat
* @lucene.experimental
*/
public class Lucene40SegmentInfosWriter extends SegmentInfosWriter {
@Override
public IndexOutput writeInfos(Directory dir, String segmentFileName, String codecID, SegmentInfos infos, IOContext context)
throws IOException {
IndexOutput out = createOutput(dir, segmentFileName, new IOContext(new FlushInfo(infos.size(), infos.totalDocCount())));
boolean success = false;
try {
/*
* TODO its not ideal that we write the format and the codecID inside the
* codec private classes but we read it in SegmentInfos.
*/
out.writeInt(SegmentInfos.FORMAT_CURRENT); // write FORMAT
out.writeString(codecID); // write codecID
out.writeLong(infos.version);
out.writeInt(infos.counter); // write counter
out.writeInt(infos.size()); // write infos
for (SegmentInfo si : infos) {
writeInfo(out, si);
}
out.writeStringStringMap(infos.getUserData());
success = true;
return out;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(out);
}
}
}
/** Save a single segment's info. */
private void writeInfo(IndexOutput output, SegmentInfo si) throws IOException {
assert si.getDelCount() <= si.docCount: "delCount=" + si.getDelCount() + " docCount=" + si.docCount + " segment=" + si.name;
// Write the Lucene version that created this segment, since 3.1
output.writeString(si.getVersion());
output.writeString(si.name);
output.writeInt(si.docCount);
output.writeLong(si.getDelGen());
// we still need to write this in 4.0 since we can open a 3.x with shared docStores
output.writeInt(si.getDocStoreOffset());
if (si.getDocStoreOffset() != -1) {
output.writeString(si.getDocStoreSegment());
output.writeByte((byte) (si.getDocStoreIsCompoundFile() ? 1:0));
}
Map<Integer,Long> normGen = si.getNormGen();
if (normGen == null) {
output.writeInt(SegmentInfo.NO);
} else {
output.writeInt(normGen.size());
for (Entry<Integer,Long> entry : normGen.entrySet()) {
output.writeInt(entry.getKey());
output.writeLong(entry.getValue());
}
}
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
output.writeInt(si.getDelCount());
output.writeByte((byte) (si.getHasProxInternal()));
output.writeString(si.getCodec().getName());
output.writeStringStringMap(si.getDiagnostics());
output.writeByte((byte) (si.getHasVectorsInternal()));
}
protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context)
throws IOException {
IndexOutput plainOut = dir.createOutput(segmentFileName, context);
ChecksumIndexOutput out = new ChecksumIndexOutput(plainOut);
return out;
}
@Override
public void prepareCommit(IndexOutput segmentOutput) throws IOException {
((ChecksumIndexOutput)segmentOutput).prepareCommit();
}
@Override
public void finishCommit(IndexOutput out) throws IOException {
((ChecksumIndexOutput)out).finishCommit();
out.close();
}
}

View File

@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
@ -89,13 +88,8 @@ public class Lucene40StoredFieldsFormat extends StoredFieldsFormat {
} }
@Override @Override
public StoredFieldsWriter fieldsWriter(Directory directory, String segment, public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si,
IOContext context) throws IOException { IOContext context) throws IOException {
return new Lucene40StoredFieldsWriter(directory, segment, context); return new Lucene40StoredFieldsWriter(directory, si.name, context);
}
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene40StoredFieldsReader.files(info, files);
} }
} }

View File

@ -34,7 +34,6 @@ import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import java.io.Closeable; import java.io.Closeable;
import java.util.Set;
import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.*; import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.*;
@ -90,8 +89,8 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
final long indexSize = indexStream.length() - HEADER_LENGTH_IDX; final long indexSize = indexStream.length() - HEADER_LENGTH_IDX;
this.size = (int) (indexSize >> 3); this.size = (int) (indexSize >> 3);
// Verify two sources of "maxDoc" agree: // Verify two sources of "maxDoc" agree:
if (this.size != si.docCount) { if (this.size != si.getDocCount()) {
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.docCount); throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.getDocCount());
} }
numTotalDocs = (int) (indexSize >> 3); numTotalDocs = (int) (indexSize >> 3);
success = true; success = true;
@ -239,9 +238,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, "", FIELDS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(info.name, "", FIELDS_EXTENSION));
}
} }

View File

@ -23,6 +23,7 @@ import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergePolicy.MergeAbortedException; import org.apache.lucene.index.MergePolicy.MergeAbortedException;
@ -208,7 +209,7 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
} }
@Override @Override
public void finish(int numDocs) throws IOException { public void finish(FieldInfos fis, int numDocs) throws IOException {
if (HEADER_LENGTH_IDX+((long) numDocs)*8 != indexStream.getFilePointer()) if (HEADER_LENGTH_IDX+((long) numDocs)*8 != indexStream.getFilePointer())
// This is most likely a bug in Sun JRE 1.6.0_04/_05; // This is most likely a bug in Sun JRE 1.6.0_04/_05;
// we detect that the bug has struck, here, and // we detect that the bug has struck, here, and
@ -244,7 +245,7 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
reader, matchingFieldsReader, rawDocLengths); reader, matchingFieldsReader, rawDocLengths);
} }
} }
finish(docCount); finish(mergeState.fieldInfos, docCount);
return docCount; return docCount;
} }

View File

@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene40;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
@ -109,12 +108,7 @@ public class Lucene40TermVectorsFormat extends TermVectorsFormat {
} }
@Override @Override
public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException { public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
return new Lucene40TermVectorsWriter(directory, segment, context); return new Lucene40TermVectorsWriter(directory, segmentInfo.name, context);
}
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
Lucene40TermVectorsReader.files(info, files);
} }
} }

View File

@ -22,7 +22,6 @@ import java.util.Arrays;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
@ -44,7 +43,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil; import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
/** /**
* Lucene 4.0 Term Vectors reader. * Lucene 4.0 Term Vectors reader.
* <p> * <p>
@ -100,7 +98,7 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
public Lucene40TermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context) public Lucene40TermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context)
throws CorruptIndexException, IOException { throws CorruptIndexException, IOException {
final String segment = si.name; final String segment = si.name;
final int size = si.docCount; final int size = si.getDocCount();
boolean success = false; boolean success = false;
@ -256,7 +254,7 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
@Override @Override
public String next() throws IOException { public String next() throws IOException {
if (fieldNumbers != null && fieldUpto < fieldNumbers.length) { if (fieldNumbers != null && fieldUpto < fieldNumbers.length) {
return fieldInfos.fieldName(fieldNumbers[fieldUpto++]); return fieldInfos.fieldInfo(fieldNumbers[fieldUpto++]).name;
} else { } else {
return null; return null;
} }
@ -264,7 +262,7 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
@Override @Override
public Terms terms() throws IOException { public Terms terms() throws IOException {
return TVFields.this.terms(fieldInfos.fieldName(fieldNumbers[fieldUpto-1])); return TVFields.this.terms(fieldInfos.fieldInfo(fieldNumbers[fieldUpto-1]).name);
} }
}; };
} }
@ -690,13 +688,5 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs); return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs);
} }
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));
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
}
}
} }

View File

@ -23,6 +23,7 @@ import java.util.Comparator;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter; import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields; import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.MergePolicy.MergeAbortedException; import org.apache.lucene.index.MergePolicy.MergeAbortedException;
@ -270,7 +271,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
numDocs += copyVectorsNoDeletions(mergeState, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2); numDocs += copyVectorsNoDeletions(mergeState, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
} }
} }
finish(numDocs); finish(mergeState.fieldInfos, numDocs);
return numDocs; return numDocs;
} }
@ -361,7 +362,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
} }
@Override @Override
public void finish(int numDocs) throws IOException { public void finish(FieldInfos fis, int numDocs) throws IOException {
if (HEADER_LENGTH_INDEX+((long) numDocs)*16 != tvx.getFilePointer()) if (HEADER_LENGTH_INDEX+((long) numDocs)*16 != tvx.getFilePointer())
// This is most likely a bug in Sun JRE 1.6.0_04/_05; // This is most likely a bug in Sun JRE 1.6.0_04/_05;
// we detect that the bug has struck, here, and // we detect that the bug has struck, here, and

View File

@ -137,6 +137,11 @@ its numbering.</p>
<p>Each segment index maintains the following:</p> <p>Each segment index maintains the following:</p>
<ul> <ul>
<li> <li>
{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment info}.
This contains metadata about a segment, such as the number of documents,
what files it uses,
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Field names}. {@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Field names}.
This contains the set of field names used in the index. This contains the set of field names used in the index.
</li> </li>
@ -197,8 +202,9 @@ An optional file indicating which documents are deleted.
<div> <div>
<p>All files belonging to a segment have the same name with varying extensions. <p>All files belonging to a segment have the same name with varying extensions.
The extensions correspond to the different file formats described below. When The extensions correspond to the different file formats described below. When
using the Compound File format (default in 1.4 and greater) these files are using the Compound File format (default in 1.4 and greater) these files (except
collapsed into a single .cfs file (see below for details)</p> for the Segment info file, the Lock file, and Deleted documents file) are collapsed
into a single .cfs file (see below for details)</p>
<p>Typically, all segments in an index are stored in a single directory, <p>Typically, all segments in an index are stored in a single directory,
although this is not required.</p> although this is not required.</p>
<p>As of version 2.1 (lock-less commits), file names are never re-used (there <p>As of version 2.1 (lock-less commits), file names are never re-used (there
@ -220,9 +226,9 @@ Lucene:</p>
<th>Brief Description</th> <th>Brief Description</th>
</tr> </tr>
<tr> <tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfosFormat Segments File}</td> <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
<td>segments.gen, segments_N</td> <td>segments.gen, segments_N</td>
<td>Stores information about segments</td> <td>Stores information about a commit point</td>
</tr> </tr>
<tr> <tr>
<td><a href="#Lock_File">Lock File</a></td> <td><a href="#Lock_File">Lock File</a></td>
@ -231,6 +237,11 @@ Lucene:</p>
file.</td> file.</td>
</tr> </tr>
<tr> <tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment Info}</td>
<td>.si</td>
<td>Stores metadata about a segment</td>
</tr>
<tr>
<td>{@link org.apache.lucene.store.CompoundFileDirectory Compound File}</td> <td>{@link org.apache.lucene.store.CompoundFileDirectory Compound File}</td>
<td>.cfs, .cfe</td> <td>.cfs, .cfe</td>
<td>An optional "virtual" file consisting of all the other index files for <td>An optional "virtual" file consisting of all the other index files for
@ -301,11 +312,6 @@ systems that frequently run out of file handles.</td>
<td>.del</td> <td>.del</td>
<td>Info about what files are deleted</td> <td>Info about what files are deleted</td>
</tr> </tr>
<tr>
<td>{@link org.apache.lucene.codecs.perfield.PerFieldPostingsFormat Field formats}</td>
<td>.per</td>
<td>Enables per-field {@link org.apache.lucene.codecs.PostingsFormat PostingsFormat} configuration.</td>
</tr>
</table> </table>
</div> </div>
<a name="Lock_File" id="Lock_File"></a> <a name="Lock_File" id="Lock_File"></a>

View File

@ -70,7 +70,7 @@ public abstract class DocValuesWriterBase extends PerDocConsumer {
* @see PackedInts#getReader(org.apache.lucene.store.DataInput) * @see PackedInts#getReader(org.apache.lucene.store.DataInput)
*/ */
protected DocValuesWriterBase(PerDocWriteState state, float acceptableOverheadRatio) { protected DocValuesWriterBase(PerDocWriteState state, float acceptableOverheadRatio) {
this.segmentName = state.segmentName; this.segmentName = state.segmentInfo.name;
this.bytesUsed = state.bytesUsed; this.bytesUsed = state.bytesUsed;
this.context = state.context; this.context = state.context;
this.acceptableOverheadRatio = acceptableOverheadRatio; this.acceptableOverheadRatio = acceptableOverheadRatio;

View File

@ -68,7 +68,7 @@ class FixedSortedBytesImpl {
throws IOException { throws IOException {
boolean success = false; boolean success = false;
try { try {
final MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_FIXED_SORTED, docValues, comp, mergeState.mergedDocCount); final MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_FIXED_SORTED, docValues, comp, mergeState.segmentInfo.getDocCount());
List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState.docBase, mergeState.docMaps, docValues, ctx); List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState.docBase, mergeState.docMaps, docValues, ctx);
final IndexOutput datOut = getOrCreateDataOut(); final IndexOutput datOut = getOrCreateDataOut();
datOut.writeInt(ctx.sizePerValues); datOut.writeInt(ctx.sizePerValues);

View File

@ -71,7 +71,7 @@ final class VarSortedBytesImpl {
throws IOException { throws IOException {
boolean success = false; boolean success = false;
try { try {
MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_VAR_SORTED, docValues, comp, mergeState.mergedDocCount); MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_VAR_SORTED, docValues, comp, mergeState.segmentInfo.getDocCount());
final List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState.docBase, mergeState.docMaps, docValues, ctx); final List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState.docBase, mergeState.docMaps, docValues, ctx);
IndexOutput datOut = getOrCreateDataOut(); IndexOutput datOut = getOrCreateDataOut();

View File

@ -20,7 +20,6 @@ package org.apache.lucene.codecs.memory;
import java.io.IOException; import java.io.IOException;
import java.util.Comparator; import java.util.Comparator;
import java.util.Iterator; import java.util.Iterator;
import java.util.Set;
import java.util.SortedMap; import java.util.SortedMap;
import java.util.TreeMap; import java.util.TreeMap;
@ -37,7 +36,6 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum; import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms; import org.apache.lucene.index.Terms;
@ -133,7 +131,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
lastDocID = docID; lastDocID = docID;
docCount++; docCount++;
if (field.indexOptions == IndexOptions.DOCS_ONLY) { if (field.getIndexOptions() == IndexOptions.DOCS_ONLY) {
buffer.writeVInt(delta); buffer.writeVInt(delta);
} else if (termDocFreq == 1) { } else if (termDocFreq == 1) {
buffer.writeVInt((delta<<1) | 1); buffer.writeVInt((delta<<1) | 1);
@ -149,7 +147,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
@Override @Override
public void addPosition(int pos, BytesRef payload, int startOffset, int endOffset) throws IOException { public void addPosition(int pos, BytesRef payload, int startOffset, int endOffset) throws IOException {
assert payload == null || field.storePayloads; assert payload == null || field.hasPayloads();
//System.out.println(" addPos pos=" + pos + " payload=" + payload); //System.out.println(" addPos pos=" + pos + " payload=" + payload);
@ -159,7 +157,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
int payloadLen = 0; int payloadLen = 0;
if (field.storePayloads) { if (field.hasPayloads()) {
payloadLen = payload == null ? 0 : payload.length; payloadLen = payload == null ? 0 : payload.length;
if (payloadLen != lastPayloadLen) { if (payloadLen != lastPayloadLen) {
lastPayloadLen = payloadLen; lastPayloadLen = payloadLen;
@ -172,7 +170,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
buffer.writeVInt(delta); buffer.writeVInt(delta);
} }
if (field.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) { if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
// don't use startOffset - lastEndOffset, because this creates lots of negative vints for synonyms, // don't use startOffset - lastEndOffset, because this creates lots of negative vints for synonyms,
// and the numbers aren't that much smaller anyways. // and the numbers aren't that much smaller anyways.
int offsetDelta = startOffset - lastOffset; int offsetDelta = startOffset - lastOffset;
@ -229,7 +227,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
assert buffer2.getFilePointer() == 0; assert buffer2.getFilePointer() == 0;
buffer2.writeVInt(stats.docFreq); buffer2.writeVInt(stats.docFreq);
if (field.indexOptions != IndexOptions.DOCS_ONLY) { if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
buffer2.writeVLong(stats.totalTermFreq-stats.docFreq); buffer2.writeVLong(stats.totalTermFreq-stats.docFreq);
} }
int pos = (int) buffer2.getFilePointer(); int pos = (int) buffer2.getFilePointer();
@ -260,7 +258,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
if (termCount > 0) { if (termCount > 0) {
out.writeVInt(termCount); out.writeVInt(termCount);
out.writeVInt(field.number); out.writeVInt(field.number);
if (field.indexOptions != IndexOptions.DOCS_ONLY) { if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
out.writeVLong(sumTotalTermFreq); out.writeVLong(sumTotalTermFreq);
} }
out.writeVLong(sumDocFreq); out.writeVLong(sumDocFreq);
@ -285,7 +283,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, EXTENSION); final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
final IndexOutput out = state.directory.createOutput(fileName, state.context); final IndexOutput out = state.directory.createOutput(fileName, state.context);
return new FieldsConsumer() { return new FieldsConsumer() {
@ -648,7 +646,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
if (!didDecode) { if (!didDecode) {
buffer.reset(current.output.bytes, 0, current.output.length); buffer.reset(current.output.bytes, 0, current.output.length);
docFreq = buffer.readVInt(); docFreq = buffer.readVInt();
if (field.indexOptions != IndexOptions.DOCS_ONLY) { if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
totalTermFreq = docFreq + buffer.readVLong(); totalTermFreq = docFreq + buffer.readVLong();
} else { } else {
totalTermFreq = -1; totalTermFreq = -1;
@ -697,14 +695,14 @@ public class MemoryPostingsFormat extends PostingsFormat {
decodeMetaData(); decodeMetaData();
FSTDocsEnum docsEnum; FSTDocsEnum docsEnum;
if (needsFreqs && field.indexOptions == IndexOptions.DOCS_ONLY) { if (needsFreqs && field.getIndexOptions() == IndexOptions.DOCS_ONLY) {
return null; return null;
} else if (reuse == null || !(reuse instanceof FSTDocsEnum)) { } else if (reuse == null || !(reuse instanceof FSTDocsEnum)) {
docsEnum = new FSTDocsEnum(field.indexOptions, field.storePayloads); docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads());
} else { } else {
docsEnum = (FSTDocsEnum) reuse; docsEnum = (FSTDocsEnum) reuse;
if (!docsEnum.canReuse(field.indexOptions, field.storePayloads)) { if (!docsEnum.canReuse(field.getIndexOptions(), field.hasPayloads())) {
docsEnum = new FSTDocsEnum(field.indexOptions, field.storePayloads); docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads());
} }
} }
return docsEnum.reset(current.output, liveDocs, docFreq); return docsEnum.reset(current.output, liveDocs, docFreq);
@ -713,22 +711,22 @@ public class MemoryPostingsFormat extends PostingsFormat {
@Override @Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException { public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
boolean hasOffsets = field.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; boolean hasOffsets = field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (needsOffsets && !hasOffsets) { if (needsOffsets && !hasOffsets) {
return null; // not available return null; // not available
} }
if (field.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
return null; return null;
} }
decodeMetaData(); decodeMetaData();
FSTDocsAndPositionsEnum docsAndPositionsEnum; FSTDocsAndPositionsEnum docsAndPositionsEnum;
if (reuse == null || !(reuse instanceof FSTDocsAndPositionsEnum)) { if (reuse == null || !(reuse instanceof FSTDocsAndPositionsEnum)) {
docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.storePayloads, hasOffsets); docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.hasPayloads(), hasOffsets);
} else { } else {
docsAndPositionsEnum = (FSTDocsAndPositionsEnum) reuse; docsAndPositionsEnum = (FSTDocsAndPositionsEnum) reuse;
if (!docsAndPositionsEnum.canReuse(field.storePayloads, hasOffsets)) { if (!docsAndPositionsEnum.canReuse(field.hasPayloads(), hasOffsets)) {
docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.storePayloads, hasOffsets); docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.hasPayloads(), hasOffsets);
} }
} }
//System.out.println("D&P reset this=" + this); //System.out.println("D&P reset this=" + this);
@ -797,7 +795,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
this.termCount = termCount; this.termCount = termCount;
final int fieldNumber = in.readVInt(); final int fieldNumber = in.readVInt();
field = fieldInfos.fieldInfo(fieldNumber); field = fieldInfos.fieldInfo(fieldNumber);
if (field.indexOptions != IndexOptions.DOCS_ONLY) { if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
sumTotalTermFreq = in.readVLong(); sumTotalTermFreq = in.readVLong();
} else { } else {
sumTotalTermFreq = -1; sumTotalTermFreq = -1;
@ -901,9 +899,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

@ -17,15 +17,11 @@ package org.apache.lucene.codecs.perfield;
* limitations under the License. * limitations under the License.
*/ */
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap;
import java.util.IdentityHashMap; import java.util.IdentityHashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.Map; import java.util.Map;
import java.util.ServiceLoader; // javadocs import java.util.ServiceLoader; // javadocs
import java.util.Set;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
@ -34,17 +30,9 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldsEnum; import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms; import org.apache.lucene.index.Terms;
import org.apache.lucene.store.DataOutput; // javadocs
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
/** /**
@ -56,32 +44,17 @@ import org.apache.lucene.util.IOUtils;
* This method uses Java's * This method uses Java's
* {@link ServiceLoader Service Provider Interface} to resolve format names. * {@link ServiceLoader Service Provider Interface} to resolve format names.
* <p> * <p>
* PerFieldFile format: * Files written by each posting format have an additional suffix containing the
* <ul> * format name. For example, in a per-field configuration instead of <tt>_1.prx</tt>
* <li>PerFieldFile (.per) --&gt; Header, IdToFormat, FieldToFormat</li> * filenames would look like <tt>_1_Lucene40.prx</tt>.
* <li>Header --&gt; {@link CodecUtil#checkHeader CodecHeader}</li>
* <li>IdToFormat,FieldToFormat --&gt; {@link DataOutput#writeStringStringMap(Map) Map&lt;String,String&gt;}</li>
* </ul>
* <p>Notes:</p>
* <ul>
* <li>each format is assigned an id, and files written by that posting format
* have an additional suffix containing the id. For example, in a per-field
* configuration instead of <tt>_1.prx</tt> filenames would look like
* <tt>_1_0.prx</tt>.</li>
* <li>IdToFormat is a mapping between these ids and the available formats.</li>
* <li>FieldToFormat is a mapping between field names and format names.</li>
* </ul>
* @see ServiceLoader * @see ServiceLoader
* @lucene.experimental * @lucene.experimental
*/ */
public abstract class PerFieldPostingsFormat extends PostingsFormat { public abstract class PerFieldPostingsFormat extends PostingsFormat {
public static final String PER_FIELD_EXTENSION = "per";
public static final String PER_FIELD_NAME = "PerField40"; public static final String PER_FIELD_NAME = "PerField40";
public static final int VERSION_START = 0; public static final String PER_FIELD_FORMAT_KEY = PerFieldPostingsFormat.class.getSimpleName() + ".format";
public static final int VERSION_LATEST = VERSION_START;
public PerFieldPostingsFormat() { public PerFieldPostingsFormat() {
super(PER_FIELD_NAME); super(PER_FIELD_NAME);
@ -93,28 +66,9 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
return new FieldsWriter(state); return new FieldsWriter(state);
} }
// NOTE: not private to avoid $accessN at runtime!!
static class FieldsConsumerAndID implements Closeable {
final FieldsConsumer fieldsConsumer;
final String segmentSuffix;
public FieldsConsumerAndID(FieldsConsumer fieldsConsumer, String segmentSuffix) {
this.fieldsConsumer = fieldsConsumer;
this.segmentSuffix = segmentSuffix;
}
@Override
public void close() throws IOException {
fieldsConsumer.close();
}
};
private class FieldsWriter extends FieldsConsumer { private class FieldsWriter extends FieldsConsumer {
private final Map<PostingsFormat,FieldsConsumerAndID> formats = new IdentityHashMap<PostingsFormat,FieldsConsumerAndID>(); private final Map<PostingsFormat,FieldsConsumer> formats = new IdentityHashMap<PostingsFormat,FieldsConsumer>();
/** Records all fields we wrote. */
private final Map<String,PostingsFormat> fieldToFormat = new HashMap<String,PostingsFormat>();
private final SegmentWriteState segmentWriteState; private final SegmentWriteState segmentWriteState;
@ -129,60 +83,31 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
throw new IllegalStateException("invalid null PostingsFormat for field=\"" + field.name + "\""); throw new IllegalStateException("invalid null PostingsFormat for field=\"" + field.name + "\"");
} }
assert !fieldToFormat.containsKey(field.name); String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, format.getName());
fieldToFormat.put(field.name, format); assert previousValue == null;
FieldsConsumerAndID consumerAndId = formats.get(format); FieldsConsumer consumer = formats.get(format);
if (consumerAndId == null) { if (consumer == null) {
// First time we are seeing this format; assign // First time we are seeing this format; create a new instance
// next id and init it:
final String segmentSuffix = getFullSegmentSuffix(field.name, final String segmentSuffix = getFullSegmentSuffix(field.name,
segmentWriteState.segmentSuffix, segmentWriteState.segmentSuffix,
""+formats.size()); format.getName());
consumerAndId = new FieldsConsumerAndID(format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix)), consumer = format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix));
segmentSuffix); formats.put(format, consumer);
formats.put(format, consumerAndId);
} }
return consumerAndId.fieldsConsumer.addField(field); // TODO: we should only provide the "slice" of FIS
// that this PF actually sees ... then stuff like
// .hasProx could work correctly?
// NOTE: .hasProx is already broken in the same way for the non-perfield case,
// if there is a fieldinfo with prox that has no postings, you get a 0 byte file.
return consumer.addField(field);
} }
@Override @Override
public void close() throws IOException { public void close() throws IOException {
// Close all subs // Close all subs
IOUtils.close(formats.values()); IOUtils.close(formats.values());
// Write _X.per: maps field name -> format name and
// format name -> format id
final String mapFileName = IndexFileNames.segmentFileName(segmentWriteState.segmentName, segmentWriteState.segmentSuffix, PER_FIELD_EXTENSION);
final IndexOutput out = segmentWriteState.directory.createOutput(mapFileName, segmentWriteState.context);
boolean success = false;
try {
CodecUtil.writeHeader(out, PER_FIELD_NAME, VERSION_LATEST);
// format name -> int id
out.writeVInt(formats.size());
for(Map.Entry<PostingsFormat,FieldsConsumerAndID> ent : formats.entrySet()) {
out.writeString(ent.getValue().segmentSuffix);
out.writeString(ent.getKey().getName());
}
// field name -> format name
out.writeVInt(fieldToFormat.size());
for(Map.Entry<String,PostingsFormat> ent : fieldToFormat.entrySet()) {
out.writeString(ent.getKey());
out.writeString(ent.getValue().getName());
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(out);
} else {
IOUtils.close(out);
}
}
} }
} }
@ -207,18 +132,21 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
// Read _X.per and init each format: // Read _X.per and init each format:
boolean success = false; boolean success = false;
try { try {
new VisitPerFieldFile(readState.dir, readState.segmentInfo.name, readState.segmentSuffix) { // Read field name -> format name
@Override for (FieldInfo fi : readState.fieldInfos) {
protected void visitOneFormat(String segmentSuffix, PostingsFormat postingsFormat) throws IOException { if (fi.isIndexed()) {
formats.put(postingsFormat, postingsFormat.fieldsProducer(new SegmentReadState(readState, segmentSuffix))); final String fieldName = fi.name;
final String formatName = fi.getAttribute(PER_FIELD_FORMAT_KEY);
if (formatName != null) {
// null formatName means the field is in fieldInfos, but has no postings!
PostingsFormat format = PostingsFormat.forName(formatName);
if (!formats.containsKey(format)) {
formats.put(format, format.fieldsProducer(new SegmentReadState(readState, formatName)));
}
fields.put(fieldName, formats.get(format));
}
} }
@Override
protected void visitOneField(String fieldName, PostingsFormat postingsFormat) throws IOException {
assert formats.containsKey(postingsFormat);
fields.put(fieldName, formats.get(postingsFormat));
} }
};
success = true; success = true;
} finally { } finally {
if (!success) { if (!success) {
@ -280,83 +208,6 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
return new FieldsReader(state); return new FieldsReader(state);
} }
private abstract class VisitPerFieldFile {
public VisitPerFieldFile(Directory dir, String segmentName, String outerSegmentSuffix) throws IOException {
final String mapFileName = IndexFileNames.segmentFileName(segmentName, outerSegmentSuffix, PER_FIELD_EXTENSION);
final IndexInput in = dir.openInput(mapFileName, IOContext.READONCE);
boolean success = false;
try {
CodecUtil.checkHeader(in, PER_FIELD_NAME, VERSION_START, VERSION_LATEST);
// Read format name -> format id
final int formatCount = in.readVInt();
for(int formatIDX=0;formatIDX<formatCount;formatIDX++) {
final String segmentSuffix = in.readString();
final String formatName = in.readString();
PostingsFormat postingsFormat = PostingsFormat.forName(formatName);
//System.out.println("do lookup " + formatName + " -> " + postingsFormat);
if (postingsFormat == null) {
throw new IllegalStateException("unable to lookup PostingsFormat for name=\"" + formatName + "\": got null");
}
// Better be defined, because it was defined
// during indexing:
visitOneFormat(segmentSuffix, postingsFormat);
}
// Read field name -> format name
final int fieldCount = in.readVInt();
for(int fieldIDX=0;fieldIDX<fieldCount;fieldIDX++) {
final String fieldName = in.readString();
final String formatName = in.readString();
visitOneField(fieldName, PostingsFormat.forName(formatName));
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(in);
} else {
IOUtils.close(in);
}
}
}
// This is called first, for all formats:
protected abstract void visitOneFormat(String segmentSuffix, PostingsFormat format) throws IOException;
// ... then this is called, for all fields:
protected abstract void visitOneField(String fieldName, PostingsFormat format) throws IOException;
}
@Override
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);
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 ->
// format mapping) // format mapping)

View File

@ -18,7 +18,6 @@ package org.apache.lucene.codecs.pulsing;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.BlockTreeTermsReader; import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter; import org.apache.lucene.codecs.BlockTreeTermsWriter;
@ -28,7 +27,6 @@ import org.apache.lucene.codecs.PostingsBaseFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -112,10 +110,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

@ -148,7 +148,7 @@ public class PulsingPostingsReader extends PostingsReaderBase {
PulsingTermState termState = (PulsingTermState) _termState; PulsingTermState termState = (PulsingTermState) _termState;
// if we have positions, its total TF, otherwise its computed based on docFreq. // if we have positions, its total TF, otherwise its computed based on docFreq.
long count = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 ? termState.totalTermFreq : termState.docFreq; long count = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 ? termState.totalTermFreq : termState.docFreq;
//System.out.println(" count=" + count + " threshold=" + maxPositions); //System.out.println(" count=" + count + " threshold=" + maxPositions);
if (count <= maxPositions) { if (count <= maxPositions) {
@ -179,7 +179,7 @@ public class PulsingPostingsReader extends PostingsReaderBase {
@Override @Override
public DocsEnum docs(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException { public DocsEnum docs(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
if (needsFreqs && field.indexOptions == IndexOptions.DOCS_ONLY) { if (needsFreqs && field.getIndexOptions() == IndexOptions.DOCS_ONLY) {
return null; return null;
} }
PulsingTermState termState = (PulsingTermState) _termState; PulsingTermState termState = (PulsingTermState) _termState;
@ -217,9 +217,9 @@ public class PulsingPostingsReader extends PostingsReaderBase {
@Override @Override
public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsAndPositionsEnum reuse, public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsAndPositionsEnum reuse,
boolean needsOffsets) throws IOException { boolean needsOffsets) throws IOException {
if (field.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
return null; return null;
} else if (needsOffsets && field.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) { } else if (needsOffsets && field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
return null; return null;
} }
@ -270,9 +270,9 @@ public class PulsingPostingsReader extends PostingsReaderBase {
private int payloadLength; private int payloadLength;
public PulsingDocsEnum(FieldInfo fieldInfo) { public PulsingDocsEnum(FieldInfo fieldInfo) {
indexOptions = fieldInfo.indexOptions; indexOptions = fieldInfo.getIndexOptions();
storePayloads = fieldInfo.storePayloads; storePayloads = fieldInfo.hasPayloads();
storeOffsets = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
} }
public PulsingDocsEnum reset(Bits liveDocs, PulsingTermState termState) { public PulsingDocsEnum reset(Bits liveDocs, PulsingTermState termState) {
@ -296,7 +296,7 @@ public class PulsingPostingsReader extends PostingsReaderBase {
} }
boolean canReuse(FieldInfo fieldInfo) { boolean canReuse(FieldInfo fieldInfo) {
return indexOptions == fieldInfo.indexOptions && storePayloads == fieldInfo.storePayloads; return indexOptions == fieldInfo.getIndexOptions() && storePayloads == fieldInfo.hasPayloads();
} }
@Override @Override
@ -400,13 +400,13 @@ public class PulsingPostingsReader extends PostingsReaderBase {
private boolean payloadRetrieved; private boolean payloadRetrieved;
public PulsingDocsAndPositionsEnum(FieldInfo fieldInfo) { public PulsingDocsAndPositionsEnum(FieldInfo fieldInfo) {
indexOptions = fieldInfo.indexOptions; indexOptions = fieldInfo.getIndexOptions();
storePayloads = fieldInfo.storePayloads; storePayloads = fieldInfo.hasPayloads();
storeOffsets = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
} }
boolean canReuse(FieldInfo fieldInfo) { boolean canReuse(FieldInfo fieldInfo) {
return indexOptions == fieldInfo.indexOptions && storePayloads == fieldInfo.storePayloads; return indexOptions == fieldInfo.getIndexOptions() && storePayloads == fieldInfo.hasPayloads();
} }
public PulsingDocsAndPositionsEnum reset(Bits liveDocs, PulsingTermState termState) { public PulsingDocsAndPositionsEnum reset(Bits liveDocs, PulsingTermState termState) {

View File

@ -124,9 +124,9 @@ public final class PulsingPostingsWriter extends PostingsWriterBase {
// our parent calls setField whenever the field changes // our parent calls setField whenever the field changes
@Override @Override
public void setField(FieldInfo fieldInfo) { public void setField(FieldInfo fieldInfo) {
this.indexOptions = fieldInfo.indexOptions; this.indexOptions = fieldInfo.getIndexOptions();
if (DEBUG) System.out.println("PW field=" + fieldInfo.name + " indexOptions=" + indexOptions); if (DEBUG) System.out.println("PW field=" + fieldInfo.name + " indexOptions=" + indexOptions);
storePayloads = fieldInfo.storePayloads; storePayloads = fieldInfo.hasPayloads();
wrappedPostingsWriter.setField(fieldInfo); wrappedPostingsWriter.setField(fieldInfo);
//DEBUG = BlockTreeTermsWriter.DEBUG; //DEBUG = BlockTreeTermsWriter.DEBUG;
} }

View File

@ -18,31 +18,22 @@ package org.apache.lucene.codecs.sep;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase; import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
/** /**
* Implementation of PerDocConsumer that uses separate files. * Implementation of PerDocConsumer that uses separate files.
* @lucene.experimental * @lucene.experimental
*/ */
public class SepDocValuesConsumer extends DocValuesWriterBase { public class SepDocValuesConsumer extends DocValuesWriterBase {
private final Directory directory; private final Directory directory;
private final FieldInfos fieldInfos;
public SepDocValuesConsumer(PerDocWriteState state) throws IOException { public SepDocValuesConsumer(PerDocWriteState state) throws IOException {
super(state); super(state);
this.directory = state.directory; this.directory = state.directory;
fieldInfos = state.fieldInfos;
} }
@Override @Override
@ -50,61 +41,9 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
return directory; return directory;
} }
public static void files(SegmentInfo segmentInfo,
Set<String> files) throws IOException {
files(segmentInfo.dir, segmentInfo.getFieldInfos(), segmentInfo.name, files);
}
@SuppressWarnings("fallthrough")
private static void files(Directory dir,FieldInfos fieldInfos, String segmentName, Set<String> files) {
for (FieldInfo fieldInfo : fieldInfos) {
if (fieldInfo.hasDocValues()) {
String filename = PerDocProducerBase.docValuesId(segmentName, fieldInfo.number);
switch (fieldInfo.getDocValuesType()) {
case BYTES_FIXED_DEREF:
case BYTES_VAR_DEREF:
case BYTES_VAR_STRAIGHT:
case BYTES_FIXED_SORTED:
case BYTES_VAR_SORTED:
files.add(IndexFileNames.segmentFileName(filename, "",
INDEX_EXTENSION));
try {
assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
INDEX_EXTENSION));
} catch (IOException e) {
// don't throw checked exception - dir is only used in assert
throw new RuntimeException(e);
}
// until here all types use an index
case BYTES_FIXED_STRAIGHT:
case FLOAT_32:
case FLOAT_64:
case VAR_INTS:
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
files.add(IndexFileNames.segmentFileName(filename, "",
DATA_EXTENSION));
try {
assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
DATA_EXTENSION));
} catch (IOException e) {
// don't throw checked exception - dir is only used in assert
throw new RuntimeException(e);
}
break;
default:
assert false;
}
}
}
}
@Override @Override
public void abort() { public void abort() {
Set<String> files = new HashSet<String>(); // We don't have to remove files here: IndexFileDeleter
files(directory, fieldInfos, segmentName, files); // will do so
IOUtils.deleteFilesIgnoringExceptions(directory, files.toArray(new String[0]));
} }
} }

View File

@ -45,7 +45,7 @@ public class SepDocValuesProducer extends PerDocProducerBase {
* {@link DocValues} instances for this segment and codec. * {@link DocValues} instances for this segment and codec.
*/ */
public SepDocValuesProducer(SegmentReadState state) throws IOException { public SepDocValuesProducer(SegmentReadState state) throws IOException {
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, state.dir, state.context); docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.getDocCount(), state.dir, state.context);
} }
@Override @Override

View File

@ -18,14 +18,14 @@ package org.apache.lucene.codecs.sep;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState; import org.apache.lucene.index.TermState;
@ -60,7 +60,7 @@ public class SepPostingsReader extends PostingsReaderBase {
int maxSkipLevels; int maxSkipLevels;
int skipMinimum; int skipMinimum;
public SepPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, String segmentSuffix) throws IOException { public SepPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, String segmentSuffix) throws IOException {
boolean success = false; boolean success = false;
try { try {
@ -69,12 +69,12 @@ public class SepPostingsReader extends PostingsReaderBase {
skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION), context); skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION), context);
if (segmentInfo.getFieldInfos().hasFreq()) { if (fieldInfos.hasFreq()) {
freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION), context); freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION), context);
} else { } else {
freqIn = null; freqIn = null;
} }
if (segmentInfo.getHasProx()) { if (fieldInfos.hasProx()) {
posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION), context); posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION), context);
payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION), context); payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION), context);
} else { } else {
@ -89,20 +89,6 @@ public class SepPostingsReader extends PostingsReaderBase {
} }
} }
public static void files(SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.DOC_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION));
if (segmentInfo.getFieldInfos().hasFreq()) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION));
}
if (segmentInfo.getHasProx()) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION));
}
}
@Override @Override
public void init(IndexInput termsIn) throws IOException { public void init(IndexInput termsIn) throws IOException {
// Make sure we are talking to the matching past writer // Make sure we are talking to the matching past writer
@ -241,13 +227,13 @@ public class SepPostingsReader extends PostingsReaderBase {
//System.out.println(" docFreq=" + termState.docFreq); //System.out.println(" docFreq=" + termState.docFreq);
termState.docIndex.read(termState.bytesReader, isFirstTerm); termState.docIndex.read(termState.bytesReader, isFirstTerm);
//System.out.println(" docIndex=" + termState.docIndex); //System.out.println(" docIndex=" + termState.docIndex);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
termState.freqIndex.read(termState.bytesReader, isFirstTerm); termState.freqIndex.read(termState.bytesReader, isFirstTerm);
if (fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { if (fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
//System.out.println(" freqIndex=" + termState.freqIndex); //System.out.println(" freqIndex=" + termState.freqIndex);
termState.posIndex.read(termState.bytesReader, isFirstTerm); termState.posIndex.read(termState.bytesReader, isFirstTerm);
//System.out.println(" posIndex=" + termState.posIndex); //System.out.println(" posIndex=" + termState.posIndex);
if (fieldInfo.storePayloads) { if (fieldInfo.hasPayloads()) {
if (isFirstTerm) { if (isFirstTerm) {
termState.payloadFP = termState.bytesReader.readVLong(); termState.payloadFP = termState.bytesReader.readVLong();
} else { } else {
@ -273,7 +259,7 @@ public class SepPostingsReader extends PostingsReaderBase {
@Override @Override
public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException { public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
if (needsFreqs && fieldInfo.indexOptions == IndexOptions.DOCS_ONLY) { if (needsFreqs && fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY) {
return null; return null;
} }
final SepTermState termState = (SepTermState) _termState; final SepTermState termState = (SepTermState) _termState;
@ -298,7 +284,7 @@ public class SepPostingsReader extends PostingsReaderBase {
DocsAndPositionsEnum reuse, boolean needsOffsets) DocsAndPositionsEnum reuse, boolean needsOffsets)
throws IOException { throws IOException {
if (fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
return null; return null;
} }
@ -306,7 +292,7 @@ public class SepPostingsReader extends PostingsReaderBase {
return null; return null;
} }
assert fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS; assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
final SepTermState termState = (SepTermState) _termState; final SepTermState termState = (SepTermState) _termState;
SepDocsAndPositionsEnum postingsEnum; SepDocsAndPositionsEnum postingsEnum;
if (reuse == null || !(reuse instanceof SepDocsAndPositionsEnum)) { if (reuse == null || !(reuse instanceof SepDocsAndPositionsEnum)) {
@ -371,9 +357,9 @@ public class SepPostingsReader extends PostingsReaderBase {
SepDocsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException { SepDocsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException {
this.liveDocs = liveDocs; this.liveDocs = liveDocs;
this.indexOptions = fieldInfo.indexOptions; this.indexOptions = fieldInfo.getIndexOptions();
omitTF = indexOptions == IndexOptions.DOCS_ONLY; omitTF = indexOptions == IndexOptions.DOCS_ONLY;
storePayloads = fieldInfo.storePayloads; storePayloads = fieldInfo.hasPayloads();
// TODO: can't we only do this if consumer // TODO: can't we only do this if consumer
// skipped consuming the previous docs? // skipped consuming the previous docs?
@ -536,7 +522,7 @@ public class SepPostingsReader extends PostingsReaderBase {
SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException { SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException {
this.liveDocs = liveDocs; this.liveDocs = liveDocs;
storePayloads = fieldInfo.storePayloads; storePayloads = fieldInfo.hasPayloads();
//System.out.println("Sep D&P init"); //System.out.println("Sep D&P init");
// TODO: can't we only do this if consumer // TODO: can't we only do this if consumer

View File

@ -115,34 +115,34 @@ public final class SepPostingsWriter extends PostingsWriterBase {
try { try {
this.skipInterval = skipInterval; this.skipInterval = skipInterval;
this.skipMinimum = skipInterval; /* set to the same for now */ this.skipMinimum = skipInterval; /* set to the same for now */
final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, DOC_EXTENSION); final String docFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, DOC_EXTENSION);
docOut = factory.createOutput(state.directory, docFileName, state.context); docOut = factory.createOutput(state.directory, docFileName, state.context);
docIndex = docOut.index(); docIndex = docOut.index();
if (state.fieldInfos.hasFreq()) { if (state.fieldInfos.hasFreq()) {
final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, FREQ_EXTENSION); final String frqFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FREQ_EXTENSION);
freqOut = factory.createOutput(state.directory, frqFileName, state.context); freqOut = factory.createOutput(state.directory, frqFileName, state.context);
freqIndex = freqOut.index(); freqIndex = freqOut.index();
} }
if (state.fieldInfos.hasProx()) { if (state.fieldInfos.hasProx()) {
final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, POS_EXTENSION); final String posFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, POS_EXTENSION);
posOut = factory.createOutput(state.directory, posFileName, state.context); posOut = factory.createOutput(state.directory, posFileName, state.context);
posIndex = posOut.index(); posIndex = posOut.index();
// TODO: -- only if at least one field stores payloads? // TODO: -- only if at least one field stores payloads?
final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, PAYLOAD_EXTENSION); final String payloadFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, PAYLOAD_EXTENSION);
payloadOut = state.directory.createOutput(payloadFileName, state.context); payloadOut = state.directory.createOutput(payloadFileName, state.context);
} }
final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, SKIP_EXTENSION); final String skipFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SKIP_EXTENSION);
skipOut = state.directory.createOutput(skipFileName, state.context); skipOut = state.directory.createOutput(skipFileName, state.context);
totalNumDocs = state.numDocs; totalNumDocs = state.segmentInfo.getDocCount();
skipListWriter = new SepSkipListWriter(skipInterval, skipListWriter = new SepSkipListWriter(skipInterval,
maxSkipLevels, maxSkipLevels,
state.numDocs, totalNumDocs,
freqOut, docOut, freqOut, docOut,
posOut, payloadOut); posOut, payloadOut);
@ -187,12 +187,12 @@ public final class SepPostingsWriter extends PostingsWriterBase {
@Override @Override
public void setField(FieldInfo fieldInfo) { public void setField(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo; this.fieldInfo = fieldInfo;
this.indexOptions = fieldInfo.indexOptions; this.indexOptions = fieldInfo.getIndexOptions();
if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) { if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
throw new UnsupportedOperationException("this codec cannot index offsets"); throw new UnsupportedOperationException("this codec cannot index offsets");
} }
skipListWriter.setIndexOptions(indexOptions); skipListWriter.setIndexOptions(indexOptions);
storePayloads = indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS && fieldInfo.storePayloads; storePayloads = indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS && fieldInfo.hasPayloads();
} }
/** Adds a new doc in this term. If this returns null /** Adds a new doc in this term. If this returns null

View File

@ -23,7 +23,7 @@ import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.TermVectorsFormat;
@ -36,7 +36,7 @@ import org.apache.lucene.codecs.TermVectorsFormat;
public final class SimpleTextCodec extends Codec { public final class SimpleTextCodec extends Codec {
private final PostingsFormat postings = new SimpleTextPostingsFormat(); private final PostingsFormat postings = new SimpleTextPostingsFormat();
private final StoredFieldsFormat storedFields = new SimpleTextStoredFieldsFormat(); private final StoredFieldsFormat storedFields = new SimpleTextStoredFieldsFormat();
private final SegmentInfosFormat segmentInfos = new SimpleTextSegmentInfosFormat(); private final SegmentInfoFormat segmentInfos = new SimpleTextSegmentInfoFormat();
private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat(); private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat(); private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
// TODO: need a plain-text impl // TODO: need a plain-text impl
@ -75,7 +75,7 @@ public final class SimpleTextCodec extends Codec {
} }
@Override @Override
public SegmentInfosFormat segmentInfosFormat() { public SegmentInfoFormat segmentInfoFormat() {
return segmentInfos; return segmentInfos;
} }

View File

@ -286,7 +286,4 @@ public class SimpleTextDocValuesConsumer extends DocValuesConsumer {
protected Type getType() { protected Type getType() {
return type; return type;
} }
} }

View File

@ -16,14 +16,13 @@ package org.apache.lucene.codecs.simpletext;
* License for the specific language governing permissions and limitations under * License for the specific language governing permissions and limitations under
* the License. * the License.
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PerDocConsumer; import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer; import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
@ -49,10 +48,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

@ -18,12 +18,10 @@ package org.apache.lucene.codecs.simpletext;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosReader; import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter; import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.SegmentInfo;
/** /**
* plaintext field infos format * plaintext field infos format
@ -44,9 +42,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

@ -18,14 +18,15 @@ package org.apache.lucene.codecs.simpletext;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set; import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.FieldInfosReader; import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValues;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
@ -51,10 +52,6 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
IndexInput input = directory.openInput(fileName, iocontext); IndexInput input = directory.openInput(fileName, iocontext);
BytesRef scratch = new BytesRef(); BytesRef scratch = new BytesRef();
boolean hasVectors = false;
boolean hasFreq = false;
boolean hasProx = false;
try { try {
SimpleTextUtil.readLine(input, scratch); SimpleTextUtil.readLine(input, scratch);
@ -97,25 +94,35 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
String dvType = readString(DOCVALUES.length, scratch); String dvType = readString(DOCVALUES.length, scratch);
final DocValues.Type docValuesType = docValuesType(dvType); final DocValues.Type docValuesType = docValuesType(dvType);
SimpleTextUtil.readLine(input, scratch); SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, INDEXOPTIONS); assert StringHelper.startsWith(scratch, INDEXOPTIONS);
IndexOptions indexOptions = IndexOptions.valueOf(readString(INDEXOPTIONS.length, scratch)); IndexOptions indexOptions = IndexOptions.valueOf(readString(INDEXOPTIONS.length, scratch));
hasVectors |= storeTermVector; SimpleTextUtil.readLine(input, scratch);
hasProx |= isIndexed && indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; assert StringHelper.startsWith(scratch, NUM_ATTS);
hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY; int numAtts = Integer.parseInt(readString(NUM_ATTS.length, scratch));
Map<String,String> atts = new HashMap<String,String>();
for (int j = 0; j < numAtts; j++) {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, ATT_KEY);
String key = readString(ATT_KEY.length, scratch);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, ATT_VALUE);
String value = readString(ATT_VALUE.length, scratch);
atts.put(key, value);
}
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, docValuesType, normsType); omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(atts));
} }
if (input.getFilePointer() != input.length()) { if (input.getFilePointer() != input.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
} }
return new FieldInfos(infos, hasFreq, hasProx, hasVectors); return new FieldInfos(infos);
} finally { } finally {
input.close(); input.close();
} }
@ -132,8 +139,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

@ -17,6 +17,7 @@ package org.apache.lucene.codecs.simpletext;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Map;
import org.apache.lucene.codecs.FieldInfosWriter; import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValues;
@ -52,6 +53,9 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
static final BytesRef NORMS_TYPE = new BytesRef(" norms type "); static final BytesRef NORMS_TYPE = new BytesRef(" norms type ");
static final BytesRef DOCVALUES = new BytesRef(" doc values "); static final BytesRef DOCVALUES = new BytesRef(" doc values ");
static final BytesRef INDEXOPTIONS = new BytesRef(" index options "); static final BytesRef INDEXOPTIONS = new BytesRef(" index options ");
static final BytesRef NUM_ATTS = new BytesRef(" attributes ");
final static BytesRef ATT_KEY = new BytesRef(" key ");
final static BytesRef ATT_VALUE = new BytesRef(" value ");
@Override @Override
public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException { public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
@ -64,7 +68,7 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
for (FieldInfo fi : infos) { for (FieldInfo fi : infos) {
assert fi.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.storePayloads; assert fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
SimpleTextUtil.write(out, NAME); SimpleTextUtil.write(out, NAME);
SimpleTextUtil.write(out, fi.name, scratch); SimpleTextUtil.write(out, fi.name, scratch);
@ -75,19 +79,19 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, ISINDEXED); SimpleTextUtil.write(out, ISINDEXED);
SimpleTextUtil.write(out, Boolean.toString(fi.isIndexed), scratch); SimpleTextUtil.write(out, Boolean.toString(fi.isIndexed()), scratch);
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, STORETV); SimpleTextUtil.write(out, STORETV);
SimpleTextUtil.write(out, Boolean.toString(fi.storeTermVector), scratch); SimpleTextUtil.write(out, Boolean.toString(fi.hasVectors()), scratch);
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, PAYLOADS); SimpleTextUtil.write(out, PAYLOADS);
SimpleTextUtil.write(out, Boolean.toString(fi.storePayloads), scratch); SimpleTextUtil.write(out, Boolean.toString(fi.hasPayloads()), scratch);
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, NORMS); SimpleTextUtil.write(out, NORMS);
SimpleTextUtil.write(out, Boolean.toString(!fi.omitNorms), scratch); SimpleTextUtil.write(out, Boolean.toString(!fi.omitsNorms()), scratch);
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, NORMS_TYPE); SimpleTextUtil.write(out, NORMS_TYPE);
@ -99,8 +103,26 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, INDEXOPTIONS); SimpleTextUtil.write(out, INDEXOPTIONS);
SimpleTextUtil.write(out, fi.indexOptions.toString(), scratch); SimpleTextUtil.write(out, fi.getIndexOptions().toString(), scratch);
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
Map<String,String> atts = fi.attributes();
int numAtts = atts == null ? 0 : atts.size();
SimpleTextUtil.write(out, NUM_ATTS);
SimpleTextUtil.write(out, Integer.toString(numAtts), scratch);
SimpleTextUtil.writeNewline(out);
if (numAtts > 0) {
for (Map.Entry<String,String> entry : atts.entrySet()) {
SimpleTextUtil.write(out, ATT_KEY);
SimpleTextUtil.write(out, entry.getKey(), scratch);
SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, ATT_VALUE);
SimpleTextUtil.write(out, entry.getValue(), scratch);
SimpleTextUtil.writeNewline(out);
}
}
} }
} finally { } finally {
out.close(); out.close();

View File

@ -514,7 +514,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
public SimpleTextTerms(String field, long termsStart) throws IOException { public SimpleTextTerms(String field, long termsStart) throws IOException {
this.termsStart = termsStart; this.termsStart = termsStart;
indexOptions = fieldInfos.fieldInfo(field).indexOptions; indexOptions = fieldInfos.fieldInfo(field).getIndexOptions();
loadTerms(); loadTerms();
} }

View File

@ -46,7 +46,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
final static BytesRef PAYLOAD = new BytesRef(" payload "); final static BytesRef PAYLOAD = new BytesRef(" payload ");
public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException { public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.segmentSuffix); final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix);
out = state.directory.createOutput(fileName, state.context); out = state.directory.createOutput(fileName, state.context);
} }
@ -107,7 +107,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
private int lastEndOffset = -1; private int lastEndOffset = -1;
public SimpleTextPostingsWriter(FieldInfo field) { public SimpleTextPostingsWriter(FieldInfo field) {
this.indexOptions = field.indexOptions; this.indexOptions = field.getIndexOptions();
writePositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; writePositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
writeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; writeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
//System.out.println("writeOffsets=" + writeOffsets); //System.out.println("writeOffsets=" + writeOffsets);

View File

@ -19,11 +19,11 @@ package org.apache.lucene.codecs.simpletext;
import java.io.IOException; import java.io.IOException;
import java.util.BitSet; import java.util.BitSet;
import java.util.Set; import java.util.Collection;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfoPerCommit;
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.IndexInput; import org.apache.lucene.store.IndexInput;
@ -63,12 +63,12 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
} }
@Override @Override
public Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) throws IOException { public Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) throws IOException {
assert info.hasDeletions(); assert info.hasDeletions();
BytesRef scratch = new BytesRef(); BytesRef scratch = new BytesRef();
CharsRef scratchUTF16 = new CharsRef(); CharsRef scratchUTF16 = new CharsRef();
String fileName = IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen()); String fileName = IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getDelGen());
IndexInput in = null; IndexInput in = null;
boolean success = false; boolean success = false;
try { try {
@ -105,12 +105,12 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
} }
@Override @Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException { public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, IOContext context) throws IOException {
BitSet set = ((SimpleTextBits) bits).bits; BitSet set = ((SimpleTextBits) bits).bits;
int size = bits.length(); int size = bits.length();
BytesRef scratch = new BytesRef(); BytesRef scratch = new BytesRef();
String fileName = IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen()); String fileName = IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getNextDelGen());
IndexOutput out = null; IndexOutput out = null;
boolean success = false; boolean success = false;
try { try {
@ -138,9 +138,9 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
} }
@Override @Override
public void files(SegmentInfo info, Set<String> files) throws IOException { public void files(SegmentInfoPerCommit info, Collection<String> files) throws IOException {
if (info.hasDeletions()) { if (info.hasDeletions()) {
files.add(IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen())); files.add(IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getDelGen()));
} }
} }

View File

@ -19,8 +19,6 @@ package org.apache.lucene.codecs.simpletext;
import java.io.IOException; import java.io.IOException;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer; import org.apache.lucene.codecs.PerDocConsumer;
@ -30,12 +28,9 @@ import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type; import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/** /**
* plain-text norms format. * plain-text norms format.
@ -49,18 +44,13 @@ public class SimpleTextNormsFormat extends NormsFormat {
@Override @Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new SimpleTextNormsPerDocConsumer(state, NORMS_SEG_SUFFIX); return new SimpleTextNormsPerDocConsumer(state);
} }
@Override @Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException { public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new SimpleTextNormsPerDocProducer(state, return new SimpleTextNormsPerDocProducer(state,
BytesRef.getUTF8SortedAsUnicodeComparator(), NORMS_SEG_SUFFIX); BytesRef.getUTF8SortedAsUnicodeComparator());
}
@Override
public void files(SegmentInfo info, Set<String> files) throws IOException {
SimpleTextNormsPerDocConsumer.files(info, files);
} }
/** /**
@ -74,8 +64,8 @@ public class SimpleTextNormsFormat extends NormsFormat {
SimpleTextPerDocProducer { SimpleTextPerDocProducer {
public SimpleTextNormsPerDocProducer(SegmentReadState state, public SimpleTextNormsPerDocProducer(SegmentReadState state,
Comparator<BytesRef> comp, String segmentSuffix) throws IOException { Comparator<BytesRef> comp) throws IOException {
super(state, comp, segmentSuffix); super(state, comp, NORMS_SEG_SUFFIX);
} }
@Override @Override
@ -105,9 +95,9 @@ public class SimpleTextNormsFormat extends NormsFormat {
public static class SimpleTextNormsPerDocConsumer extends public static class SimpleTextNormsPerDocConsumer extends
SimpleTextPerDocConsumer { SimpleTextPerDocConsumer {
public SimpleTextNormsPerDocConsumer(PerDocWriteState state, public SimpleTextNormsPerDocConsumer(PerDocWriteState state)
String segmentSuffix) throws IOException { throws IOException {
super(state, segmentSuffix); super(state, NORMS_SEG_SUFFIX);
} }
@Override @Override
@ -128,27 +118,8 @@ public class SimpleTextNormsFormat extends NormsFormat {
@Override @Override
public void abort() { public void abort() {
Set<String> files = new HashSet<String>(); // We don't have to remove files here: IndexFileDeleter
filesInternal(state.fieldInfos, state.segmentName, files, segmentSuffix); // will do so
IOUtils.deleteFilesIgnoringExceptions(state.directory,
files.toArray(new String[0]));
}
public static void files(SegmentInfo segmentInfo, Set<String> files)
throws IOException {
filesInternal(segmentInfo.getFieldInfos(), segmentInfo.name, files,
NORMS_SEG_SUFFIX);
}
public static void filesInternal(FieldInfos fieldInfos, String segmentName,
Set<String> files, String segmentSuffix) {
for (FieldInfo fieldInfo : fieldInfos) {
if (fieldInfo.hasNorms()) {
String id = docValuesId(segmentName, fieldInfo.number);
files.add(IndexFileNames.segmentFileName(id, "",
segmentSuffix));
}
}
} }
} }
} }

View File

@ -1,4 +1,5 @@
package org.apache.lucene.codecs.simpletext; package org.apache.lucene.codecs.simpletext;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this * contributor license agreements. See the NOTICE file distributed with this
@ -15,20 +16,14 @@ package org.apache.lucene.codecs.simpletext;
* License for the specific language governing permissions and limitations under * License for the specific language governing permissions and limitations under
* the License. * the License.
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.PerDocConsumer; import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.DocValues.Type; import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
/** /**
* @lucene.experimental * @lucene.experimental
@ -51,44 +46,17 @@ class SimpleTextPerDocConsumer extends PerDocConsumer {
@Override @Override
public DocValuesConsumer addValuesField(Type type, FieldInfo field) public DocValuesConsumer addValuesField(Type type, FieldInfo field)
throws IOException { throws IOException {
return new SimpleTextDocValuesConsumer(SimpleTextDocValuesFormat.docValuesId(state.segmentName, return new SimpleTextDocValuesConsumer(SimpleTextDocValuesFormat.docValuesId(state.segmentInfo.name,
field.number), state.directory, state.context, type, segmentSuffix); field.number), state.directory, state.context, type, segmentSuffix);
} }
@Override @Override
public void abort() { public void abort() {
Set<String> files = new HashSet<String>(); // We don't have to remove files here: IndexFileDeleter
files(state.directory, state.fieldInfos, state.segmentName, files, segmentSuffix); // will do so
IOUtils.deleteFilesIgnoringExceptions(state.directory,
files.toArray(new String[0]));
}
static void files(SegmentInfo info, Set<String> files, String segmentSuffix) throws IOException {
files(info.dir, info.getFieldInfos(), info.name, files, segmentSuffix);
} }
static String docValuesId(String segmentsName, int fieldId) { static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId; return segmentsName + "_" + fieldId;
} }
@SuppressWarnings("fallthrough")
private static void files(Directory dir, FieldInfos fieldInfos,
String segmentName, Set<String> files, String segmentSuffix) {
for (FieldInfo fieldInfo : fieldInfos) {
if (fieldInfo.hasDocValues()) {
String filename = docValuesId(segmentName, fieldInfo.number);
files.add(IndexFileNames.segmentFileName(filename, "",
segmentSuffix));
try {
assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
segmentSuffix));
} catch (IOException e) {
// don't throw checked exception - dir is only used in assert
throw new RuntimeException(e);
}
}
}
}
} }

View File

@ -68,7 +68,7 @@ public class SimpleTextPerDocProducer extends PerDocProducerBase {
this.segmentSuffix = segmentSuffix; this.segmentSuffix = segmentSuffix;
if (anyDocValuesFields(state.fieldInfos)) { if (anyDocValuesFields(state.fieldInfos)) {
docValues = load(state.fieldInfos, state.segmentInfo.name, docValues = load(state.fieldInfos, state.segmentInfo.name,
state.segmentInfo.docCount, state.dir, state.context); state.segmentInfo.getDocCount(), state.dir, state.context);
} else { } else {
docValues = new TreeMap<String, DocValues>(); docValues = new TreeMap<String, DocValues>();
} }

View File

@ -18,12 +18,10 @@ package org.apache.lucene.codecs.simpletext;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
@ -58,9 +56,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

@ -17,9 +17,9 @@ package org.apache.lucene.codecs.simpletext;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.codecs.SegmentInfosFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.SegmentInfosReader; import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.codecs.SegmentInfosWriter; import org.apache.lucene.codecs.SegmentInfoWriter;
/** /**
* plain text segments file format. * plain text segments file format.
@ -27,17 +27,19 @@ import org.apache.lucene.codecs.SegmentInfosWriter;
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B> * <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
* @lucene.experimental * @lucene.experimental
*/ */
public class SimpleTextSegmentInfosFormat extends SegmentInfosFormat { public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
private final SegmentInfosReader reader = new SimpleTextSegmentInfosReader(); private final SegmentInfoReader reader = new SimpleTextSegmentInfoReader();
private final SegmentInfosWriter writer = new SimpleTextSegmentInfosWriter(); private final SegmentInfoWriter writer = new SimpleTextSegmentInfoWriter();
public static final String SI_EXTENSION = "si";
@Override @Override
public SegmentInfosReader getSegmentInfosReader() { public SegmentInfoReader getSegmentInfosReader() {
return reader; return reader;
} }
@Override @Override
public SegmentInfosWriter getSegmentInfosWriter() { public SegmentInfoWriter getSegmentInfosWriter() {
return writer; return writer;
} }
} }

View File

@ -0,0 +1,127 @@
package org.apache.lucene.codecs.simpletext;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.*;
/**
* reads plaintext segments files
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
* @lucene.experimental
*/
public class SimpleTextSegmentInfoReader extends SegmentInfoReader {
@Override
public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
BytesRef scratch = new BytesRef();
String segFileName = IndexFileNames.segmentFileName(segmentName, "", SimpleTextSegmentInfoFormat.SI_EXTENSION);
IndexInput input = directory.openInput(segFileName, context);
boolean success = false;
try {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_VERSION);
final String version = readString(SI_VERSION.length, scratch);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_DOCCOUNT);
final int docCount = Integer.parseInt(readString(SI_DOCCOUNT.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_USECOMPOUND);
final boolean isCompoundFile = Boolean.parseBoolean(readString(SI_USECOMPOUND.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_NUM_DIAG);
int numDiag = Integer.parseInt(readString(SI_NUM_DIAG.length, scratch));
Map<String,String> diagnostics = new HashMap<String,String>();
for (int i = 0; i < numDiag; i++) {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_DIAG_KEY);
String key = readString(SI_DIAG_KEY.length, scratch);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_DIAG_VALUE);
String value = readString(SI_DIAG_VALUE.length, scratch);
diagnostics.put(key, value);
}
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_NUM_ATTS);
int numAtts = Integer.parseInt(readString(SI_NUM_ATTS.length, scratch));
Map<String,String> attributes = new HashMap<String,String>();
for (int i = 0; i < numAtts; i++) {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_ATT_KEY);
String key = readString(SI_ATT_KEY.length, scratch);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_ATT_VALUE);
String value = readString(SI_ATT_VALUE.length, scratch);
attributes.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,
isCompoundFile, null, diagnostics, Collections.unmodifiableMap(attributes));
info.setFiles(files);
success = true;
return info;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(input);
} else {
input.close();
}
}
}
private String readString(int offset, BytesRef scratch) {
return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
}
}

View File

@ -0,0 +1,136 @@
package org.apache.lucene.codecs.simpletext;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.SegmentInfoWriter;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* writes plaintext segments files
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
* @lucene.experimental
*/
public class SimpleTextSegmentInfoWriter extends SegmentInfoWriter {
final static BytesRef SI_VERSION = new BytesRef(" version ");
final static BytesRef SI_DOCCOUNT = new BytesRef(" number of documents ");
final static BytesRef SI_USECOMPOUND = new BytesRef(" uses compound file ");
final static BytesRef SI_NUM_DIAG = new BytesRef(" diagnostics ");
final static BytesRef SI_DIAG_KEY = new BytesRef(" key ");
final static BytesRef SI_DIAG_VALUE = new BytesRef(" value ");
final static BytesRef SI_NUM_ATTS = new BytesRef(" attributes ");
final static BytesRef SI_ATT_KEY = new BytesRef(" key ");
final static BytesRef SI_ATT_VALUE = new BytesRef(" value ");
final static BytesRef SI_NUM_FILES = new BytesRef(" files ");
final static BytesRef SI_FILE = new BytesRef(" file ");
@Override
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
String segFileName = IndexFileNames.segmentFileName(si.name, "", SimpleTextSegmentInfoFormat.SI_EXTENSION);
si.addFile(segFileName);
boolean success = false;
IndexOutput output = dir.createOutput(segFileName, ioContext);
try {
BytesRef scratch = new BytesRef();
SimpleTextUtil.write(output, SI_VERSION);
SimpleTextUtil.write(output, si.getVersion(), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_DOCCOUNT);
SimpleTextUtil.write(output, Integer.toString(si.getDocCount()), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_USECOMPOUND);
SimpleTextUtil.write(output, Boolean.toString(si.getUseCompoundFile()), scratch);
SimpleTextUtil.writeNewline(output);
Map<String,String> diagnostics = si.getDiagnostics();
int numDiagnostics = diagnostics == null ? 0 : diagnostics.size();
SimpleTextUtil.write(output, SI_NUM_DIAG);
SimpleTextUtil.write(output, Integer.toString(numDiagnostics), scratch);
SimpleTextUtil.writeNewline(output);
if (numDiagnostics > 0) {
for (Map.Entry<String,String> diagEntry : diagnostics.entrySet()) {
SimpleTextUtil.write(output, SI_DIAG_KEY);
SimpleTextUtil.write(output, diagEntry.getKey(), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_DIAG_VALUE);
SimpleTextUtil.write(output, diagEntry.getValue(), scratch);
SimpleTextUtil.writeNewline(output);
}
}
Map<String,String> atts = si.attributes();
int numAtts = atts == null ? 0 : atts.size();
SimpleTextUtil.write(output, SI_NUM_ATTS);
SimpleTextUtil.write(output, Integer.toString(numAtts), scratch);
SimpleTextUtil.writeNewline(output);
if (numAtts > 0) {
for (Map.Entry<String,String> entry : atts.entrySet()) {
SimpleTextUtil.write(output, SI_ATT_KEY);
SimpleTextUtil.write(output, entry.getKey(), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_ATT_VALUE);
SimpleTextUtil.write(output, entry.getValue(), scratch);
SimpleTextUtil.writeNewline(output);
}
}
Set<String> files = si.files();
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;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
} else {
output.close();
}
}
}
}

Some files were not shown because too many files have changed in this diff Show More