LUCENE-5215: Add support for FieldInfos generation

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1527154 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Shai Erera 2013-09-28 06:19:00 +00:00
parent a8d9e70b73
commit 1a740cad06
64 changed files with 1425 additions and 372 deletions

View File

@ -47,8 +47,8 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextFieldInfosWriter.*;
public class SimpleTextFieldInfosReader extends FieldInfosReader {
@Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);
BytesRef scratch = new BytesRef();
@ -104,6 +104,10 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
String dvType = readString(DOCVALUES.length, scratch);
final DocValuesType docValuesType = docValuesType(dvType);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, DOCVALUES_GEN);
final long dvGen = Long.parseLong(readString(DOCVALUES_GEN.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, NUM_ATTS);
int numAtts = Integer.parseInt(readString(NUM_ATTS.length, scratch));
@ -122,6 +126,7 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(atts));
infos[i].setDocValuesGen(dvGen);
}
if (input.getFilePointer() != input.length()) {

View File

@ -53,14 +53,15 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
static final BytesRef NORMS = new BytesRef(" norms ");
static final BytesRef NORMS_TYPE = new BytesRef(" norms type ");
static final BytesRef DOCVALUES = new BytesRef(" doc values ");
static final BytesRef DOCVALUES_GEN = new BytesRef(" doc values gen ");
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
public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
IndexOutput out = directory.createOutput(fileName, context);
BytesRef scratch = new BytesRef();
boolean success = false;
@ -109,6 +110,10 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
SimpleTextUtil.write(out, getDocValuesType(fi.getDocValuesType()), scratch);
SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, DOCVALUES_GEN);
SimpleTextUtil.write(out, Long.toString(fi.getDocValuesGen()), scratch);
SimpleTextUtil.writeNewline(out);
Map<String,String> atts = fi.attributes();
int numAtts = atts == null ? 0 : atts.size();
SimpleTextUtil.write(out, NUM_ATTS);

View File

@ -119,7 +119,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
loader.reload(classloader);
}
private static Codec defaultCodec = Codec.forName("Lucene45");
private static Codec defaultCodec = Codec.forName("Lucene46");
/** expert: returns the default codec used for newly created
* {@link IndexWriterConfig}s.

View File

@ -35,5 +35,5 @@ public abstract class FieldInfosReader {
/** Read the {@link FieldInfos} previously written with {@link
* FieldInfosWriter}. */
public abstract FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException;
public abstract FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException;
}

View File

@ -35,5 +35,5 @@ public abstract class FieldInfosWriter {
/** Writes the provided {@link FieldInfos} to the
* directory. */
public abstract void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException;
public abstract void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException;
}

View File

@ -21,13 +21,13 @@ package org.apache.lucene.codecs;
* A codec that forwards all its method calls to another codec.
* <p>
* Extend this class when you need to reuse the functionality of an existing
* codec. For example, if you want to build a codec that redefines Lucene45's
* codec. For example, if you want to build a codec that redefines Lucene46's
* {@link LiveDocsFormat}:
* <pre class="prettyprint">
* public final class CustomCodec extends FilterCodec {
*
* public CustomCodec() {
* super("CustomCodec", new Lucene45Codec());
* super("CustomCodec", new Lucene46Codec());
* }
*
* public LiveDocsFormat liveDocsFormat() {

View File

@ -49,7 +49,7 @@ class Lucene40FieldInfosReader extends FieldInfosReader {
}
@Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosFormat.FIELD_INFOS_EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);

View File

@ -95,7 +95,7 @@ public class Lucene42Codec extends Codec {
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
public FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}

View File

@ -83,10 +83,11 @@ import org.apache.lucene.store.DataOutput; // javadoc
* </ul>
*
* @lucene.experimental
* @deprecated Only for reading old 4.2-4.5 segments
*/
public final class Lucene42FieldInfosFormat extends FieldInfosFormat {
@Deprecated
public class Lucene42FieldInfosFormat extends FieldInfosFormat {
private final FieldInfosReader reader = new Lucene42FieldInfosReader();
private final FieldInfosWriter writer = new Lucene42FieldInfosWriter();
/** Sole constructor. */
public Lucene42FieldInfosFormat() {
@ -99,7 +100,7 @@ public final class Lucene42FieldInfosFormat extends FieldInfosFormat {
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
return writer;
throw new UnsupportedOperationException("this codec can only be used for reading");
}
/** Extension of field infos */

View File

@ -38,8 +38,10 @@ import org.apache.lucene.util.IOUtils;
* Lucene 4.2 FieldInfos reader.
*
* @lucene.experimental
* @deprecated Only for reading old 4.2-4.5 segments
* @see Lucene42FieldInfosFormat
*/
@Deprecated
final class Lucene42FieldInfosReader extends FieldInfosReader {
/** Sole constructor. */
@ -47,7 +49,7 @@ final class Lucene42FieldInfosReader extends FieldInfosReader {
}
@Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene42FieldInfosFormat.EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);

View File

@ -45,10 +45,12 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
*
* @see org.apache.lucene.codecs.lucene45 package documentation for file format details.
* @lucene.experimental
* @deprecated Only for reading old 4.3-4.5 segments
*/
// NOTE: if we make largish changes in a minor release, easier to just make Lucene46Codec or whatever
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
@Deprecated
public class Lucene45Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
@ -92,7 +94,7 @@ public class Lucene45Codec extends Codec {
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
public FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}

View File

@ -0,0 +1,139 @@
package org.apache.lucene.codecs.lucene46;
/*
* 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;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 4.6 index format, with configurable per-field postings
* and docvalues formats.
* <p>
* If you want to reuse functionality of this codec in another codec, extend
* {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene46 package documentation for file format details.
* @lucene.experimental
*/
// NOTE: if we make largish changes in a minor release, easier to just make Lucene46Codec or whatever
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
public class Lucene46Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return Lucene46Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Lucene46Codec.this.getDocValuesFormatForField(field);
}
};
/** Sole constructor. */
public Lucene46Codec() {
super("Lucene46");
}
@Override
public final StoredFieldsFormat storedFieldsFormat() {
return fieldsFormat;
}
@Override
public final TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@Override
public final SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene41"
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultFormat;
}
/** Returns the docvalues format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene45"
*/
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene45");
private final NormsFormat normsFormat = new Lucene42NormsFormat();
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -0,0 +1,126 @@
package org.apache.lucene.codecs.lucene46;
/*
* 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.CodecUtil;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.DataOutput;
/**
* Lucene 4.6 Field Infos format.
* <p>
* <p>Field names are stored in the field info file, with suffix <tt>.fnm</tt>.</p>
* <p>FieldInfos (.fnm) --&gt; Header,FieldsCount, &lt;FieldName,FieldNumber,
* FieldBits,DocValuesBits,DocValuesGen,Attributes&gt; <sup>FieldsCount</sup></p>
* <p>Data types:
* <ul>
* <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>FieldBits, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>FieldNumber --&gt; {@link DataOutput#writeInt VInt}</li>
* <li>Attributes --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
* <li>DocValuesGen --&gt; {@link DataOutput#writeLong(long) Int64}</li>
* </ul>
* </p>
* Field Descriptions:
* <ul>
* <li>FieldsCount: the number of fields in this file.</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
* Lucene, the fields are not numbered implicitly by their order in the
* file, instead explicitly.</li>
* <li>FieldBits: a byte containing field options.
* <ul>
* <li>The low-order bit is one for indexed fields, and zero for non-indexed
* fields.</li>
* <li>The second lowest-order bit is one for fields that have term vectors
* stored, and zero for fields without term vectors.</li>
* <li>If the third lowest order-bit is set (0x4), offsets are stored into
* the postings list in addition to positions.</li>
* <li>Fourth bit is unused.</li>
* <li>If the fifth lowest-order bit is set (0x10), norms are omitted for the
* indexed field.</li>
* <li>If the sixth lowest-order bit is set (0x20), payloads are stored for the
* indexed field.</li>
* <li>If the seventh lowest-order bit is set (0x40), term frequencies and
* positions omitted for the indexed field.</li>
* <li>If the eighth lowest-order bit is set (0x80), positions are omitted for the
* indexed field.</li>
* </ul>
* </li>
* <li>DocValuesBits: a byte containing per-document value types. The type
* recorded as two four-bit integers, with the high-order bits representing
* <code>norms</code> options, and the low-order bits representing
* {@code DocValues} options. Each four-bit integer can be decoded as such:
* <ul>
* <li>0: no DocValues for this field.</li>
* <li>1: NumericDocValues. ({@link DocValuesType#NUMERIC})</li>
* <li>2: BinaryDocValues. ({@code DocValuesType#BINARY})</li>
* <li>3: SortedDocValues. ({@code DocValuesType#SORTED})</li>
* </ul>
* </li>
* <li>DocValuesGen is the generation count of the field's DocValues. If this is -1,
* there are no DocValues updates to that field. Anything above zero means there
* are updates stored by {@link DocValuesFormat}.</li>
* <li>Attributes: a key-value map of codec-private attributes.</li>
* </ul>
*
* @lucene.experimental
*/
public final class Lucene46FieldInfosFormat extends FieldInfosFormat {
private final FieldInfosReader reader = new Lucene46FieldInfosReader();
private final FieldInfosWriter writer = new Lucene46FieldInfosWriter();
/** Sole constructor. */
public Lucene46FieldInfosFormat() {
}
@Override
public FieldInfosReader getFieldInfosReader() throws IOException {
return reader;
}
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
return writer;
}
/** Extension of field infos */
static final String EXTENSION = "fnm";
// Codec header
static final String CODEC_NAME = "Lucene46FieldInfos";
static final int FORMAT_START = 0;
static final int FORMAT_CURRENT = FORMAT_START;
// Field flags
static final byte IS_INDEXED = 0x1;
static final byte STORE_TERMVECTOR = 0x2;
static final byte STORE_OFFSETS_IN_POSTINGS = 0x4;
static final byte OMIT_NORMS = 0x10;
static final byte STORE_PAYLOADS = 0x20;
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
static final byte OMIT_POSITIONS = -128;
}

View File

@ -0,0 +1,125 @@
package org.apache.lucene.codecs.lucene46;
/*
* 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 org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.6 FieldInfos reader.
*
* @lucene.experimental
* @see Lucene46FieldInfosFormat
*/
final class Lucene46FieldInfosReader extends FieldInfosReader {
/** Sole constructor. */
public Lucene46FieldInfosReader() {
}
@Override
public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, Lucene46FieldInfosFormat.EXTENSION);
IndexInput input = directory.openInput(fileName, context);
boolean success = false;
try {
CodecUtil.checkHeader(input, Lucene46FieldInfosFormat.CODEC_NAME,
Lucene46FieldInfosFormat.FORMAT_START,
Lucene46FieldInfosFormat.FORMAT_CURRENT);
final int size = input.readVInt(); //read in the size
FieldInfo infos[] = new FieldInfo[size];
for (int i = 0; i < size; i++) {
String name = input.readString();
final int fieldNumber = input.readVInt();
byte bits = input.readByte();
boolean isIndexed = (bits & Lucene46FieldInfosFormat.IS_INDEXED) != 0;
boolean storeTermVector = (bits & Lucene46FieldInfosFormat.STORE_TERMVECTOR) != 0;
boolean omitNorms = (bits & Lucene46FieldInfosFormat.OMIT_NORMS) != 0;
boolean storePayloads = (bits & Lucene46FieldInfosFormat.STORE_PAYLOADS) != 0;
final IndexOptions indexOptions;
if (!isIndexed) {
indexOptions = null;
} else if ((bits & Lucene46FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_ONLY;
} else if ((bits & Lucene46FieldInfosFormat.OMIT_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS;
} else if ((bits & Lucene46FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
} else {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
}
// DV Types are packed in one byte
byte val = input.readByte();
final DocValuesType docValuesType = getDocValuesType(input, (byte) (val & 0x0F));
final DocValuesType normsType = getDocValuesType(input, (byte) ((val >>> 4) & 0x0F));
final long dvGen = input.readLong();
final Map<String,String> attributes = input.readStringStringMap();
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes));
infos[i].setDocValuesGen(dvGen);
}
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 + ")");
}
FieldInfos fieldInfos = new FieldInfos(infos);
success = true;
return fieldInfos;
} finally {
if (success) {
input.close();
} else {
IOUtils.closeWhileHandlingException(input);
}
}
}
private static DocValuesType getDocValuesType(IndexInput input, byte b) throws IOException {
if (b == 0) {
return null;
} else if (b == 1) {
return DocValuesType.NUMERIC;
} else if (b == 2) {
return DocValuesType.BINARY;
} else if (b == 3) {
return DocValuesType.SORTED;
} else if (b == 4) {
return DocValuesType.SORTED_SET;
} else {
throw new CorruptIndexException("invalid docvalues byte: " + b + " (resource=" + input + ")");
}
}
}

View File

@ -0,0 +1,109 @@
package org.apache.lucene.codecs.lucene46;
/*
* 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.CodecUtil;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.FieldInfo.DocValuesType;
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.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.6 FieldInfos writer.
*
* @see Lucene46FieldInfosFormat
* @lucene.experimental
*/
final class Lucene46FieldInfosWriter extends FieldInfosWriter {
/** Sole constructor. */
public Lucene46FieldInfosWriter() {
}
@Override
public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, Lucene46FieldInfosFormat.EXTENSION);
IndexOutput output = directory.createOutput(fileName, context);
boolean success = false;
try {
CodecUtil.writeHeader(output, Lucene46FieldInfosFormat.CODEC_NAME, Lucene46FieldInfosFormat.FORMAT_CURRENT);
output.writeVInt(infos.size());
for (FieldInfo fi : infos) {
IndexOptions indexOptions = fi.getIndexOptions();
byte bits = 0x0;
if (fi.hasVectors()) bits |= Lucene46FieldInfosFormat.STORE_TERMVECTOR;
if (fi.omitsNorms()) bits |= Lucene46FieldInfosFormat.OMIT_NORMS;
if (fi.hasPayloads()) bits |= Lucene46FieldInfosFormat.STORE_PAYLOADS;
if (fi.isIndexed()) {
bits |= Lucene46FieldInfosFormat.IS_INDEXED;
assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
if (indexOptions == IndexOptions.DOCS_ONLY) {
bits |= Lucene46FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS;
} else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
bits |= Lucene46FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS;
} else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
bits |= Lucene46FieldInfosFormat.OMIT_POSITIONS;
}
}
output.writeString(fi.name);
output.writeVInt(fi.number);
output.writeByte(bits);
// pack the DV types in one byte
final byte dv = docValuesByte(fi.getDocValuesType());
final byte nrm = docValuesByte(fi.getNormType());
assert (dv & (~0xF)) == 0 && (nrm & (~0x0F)) == 0;
byte val = (byte) (0xff & ((nrm << 4) | dv));
output.writeByte(val);
output.writeLong(fi.getDocValuesGen());
output.writeStringStringMap(fi.attributes());
}
success = true;
} finally {
if (success) {
output.close();
} else {
IOUtils.closeWhileHandlingException(output);
}
}
}
private static byte docValuesByte(DocValuesType type) {
if (type == null) {
return 0;
} else if (type == DocValuesType.NUMERIC) {
return 1;
} else if (type == DocValuesType.BINARY) {
return 2;
} else if (type == DocValuesType.SORTED) {
return 3;
} else if (type == DocValuesType.SORTED_SET) {
return 4;
} else {
throw new AssertionError();
}
}
}

View File

@ -0,0 +1,398 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Lucene 4.6 file format.
<h1>Apache Lucene - Index File Formats</h1>
<div>
<ul>
<li><a href="#Introduction">Introduction</a></li>
<li><a href="#Definitions">Definitions</a>
<ul>
<li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
<li><a href="#Types_of_Fields">Types of Fields</a></li>
<li><a href="#Segments">Segments</a></li>
<li><a href="#Document_Numbers">Document Numbers</a></li>
</ul>
</li>
<li><a href="#Overview">Index Structure Overview</a></li>
<li><a href="#File_Naming">File Naming</a></li>
<li><a href="#file-names">Summary of File Extensions</a></li>
<ul>
<li><a href="#Lock_File">Lock File</a></li>
<li><a href="#History">History</a></li>
<li><a href="#Limitations">Limitations</a></li>
</ul>
</ul>
</div>
<a name="Introduction"></a>
<h2>Introduction</h2>
<div>
<p>This document defines the index file formats used in this version of Lucene.
If you are using a different version of Lucene, please consult the copy of
<code>docs/</code> that was distributed with
the version you are using.</p>
<p>Apache Lucene is written in Java, but several efforts are underway to write
<a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
Lucene in other programming languages</a>. If these versions are to remain
compatible with Apache Lucene, then a language-independent definition of the
Lucene index format is required. This document thus attempts to provide a
complete and independent definition of the Apache Lucene file formats.</p>
<p>As Lucene evolves, this document should evolve. Versions of Lucene in
different programming languages should endeavor to agree on file formats, and
generate new versions of this document.</p>
</div>
<a name="Definitions" id="Definitions"></a>
<h2>Definitions</h2>
<div>
<p>The fundamental concepts in Lucene are index, document, field and term.</p>
<p>An index contains a sequence of documents.</p>
<ul>
<li>A document is a sequence of fields.</li>
<li>A field is a named sequence of terms.</li>
<li>A term is a sequence of bytes.</li>
</ul>
<p>The same sequence of bytes in two different fields is considered a different
term. Thus terms are represented as a pair: the string naming the field, and the
bytes within the field.</p>
<a name="Inverted_Indexing"></a>
<h3>Inverted Indexing</h3>
<p>The index stores statistics about terms in order to make term-based search
more efficient. Lucene's index falls into the family of indexes known as an
<i>inverted index.</i> This is because it can list, for a term, the documents
that contain it. This is the inverse of the natural relationship, in which
documents list terms.</p>
<a name="Types_of_Fields"></a>
<h3>Types of Fields</h3>
<p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
in the index literally, in a non-inverted manner. Fields that are inverted are
called <i>indexed</i>. A field may be both stored and indexed.</p>
<p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
text of a field may be used literally as a term to be indexed. Most fields are
tokenized, but sometimes it is useful for certain identifier fields to be
indexed literally.</p>
<p>See the {@link org.apache.lucene.document.Field Field}
java docs for more information on Fields.</p>
<a name="Segments" id="Segments"></a>
<h3>Segments</h3>
<p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
Each segment is a fully independent index, which could be searched separately.
Indexes evolve by:</p>
<ol>
<li>Creating new segments for newly added documents.</li>
<li>Merging existing segments.</li>
</ol>
<p>Searches may involve multiple segments and/or multiple indexes, each index
potentially composed of a set of segments.</p>
<a name="Document_Numbers"></a>
<h3>Document Numbers</h3>
<p>Internally, Lucene refers to documents by an integer <i>document number</i>.
The first document added to an index is numbered zero, and each subsequent
document added gets a number one greater than the previous.</p>
<p>Note that a document's number may change, so caution should be taken when
storing these numbers outside of Lucene. In particular, numbers may change in
the following situations:</p>
<ul>
<li>
<p>The numbers stored in each segment are unique only within the segment, and
must be converted before they can be used in a larger context. The standard
technique is to allocate each segment a range of values, based on the range of
numbers used in that segment. To convert a document number from a segment to an
external value, the segment's <i>base</i> document number is added. To convert
an external value back to a segment-specific value, the segment is identified
by the range that the external value is in, and the segment's base value is
subtracted. For example two five document segments might be combined, so that
the first segment has a base value of zero, and the second of five. Document
three from the second segment would have an external value of eight.</p>
</li>
<li>
<p>When documents are deleted, gaps are created in the numbering. These are
eventually removed as the index evolves through merging. Deleted documents are
dropped when segments are merged. A freshly-merged segment thus has no gaps in
its numbering.</p>
</li>
</ul>
</div>
<a name="Overview" id="Overview"></a>
<h2>Index Structure Overview</h2>
<div>
<p>Each segment index maintains the following:</p>
<ul>
<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.lucene46.Lucene46FieldInfosFormat Field names}.
This contains the set of field names used in the index.
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Stored Field values}.
This contains, for each document, a list of attribute-value pairs, where the attributes
are field names. These are used to store auxiliary information about the document, such as
its title, url, or an identifier to access a database. The set of stored fields are what is
returned for each hit when searching. This is keyed by document number.
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term dictionary}.
A dictionary containing all of the terms used in all of the
indexed fields of all of the documents. The dictionary also contains the number
of documents which contain the term, and pointers to the term's frequency and
proximity data.
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Frequency data}.
For each term in the dictionary, the numbers of all the
documents that contain that term, and the frequency of the term in that
document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Proximity data}.
For each term in the dictionary, the positions that the
term occurs in each document. Note that this will not exist if all fields in
all documents omit position data.
</li>
<li>
{@link org.apache.lucene.codecs.lucene42.Lucene42NormsFormat Normalization factors}.
For each field in each document, a value is stored
that is multiplied into the score for hits on that field.
</li>
<li>
{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vectors}.
For each field in each document, the term vector (sometimes
called document vector) may be stored. A term vector consists of term text and
term frequency. To add Term Vectors to your index see the
{@link org.apache.lucene.document.Field Field} constructors
</li>
<li>
{@link org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat Per-document values}.
Like stored values, these are also keyed by document
number, but are generally intended to be loaded into main memory for fast
access. Whereas stored values are generally intended for summary results from
searches, per-document values are useful for things like scoring factors.
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted documents}.
An optional file indicating which documents are deleted.
</li>
</ul>
<p>Details on each of these are provided in their linked pages.</p>
</div>
<a name="File_Naming"></a>
<h2>File Naming</h2>
<div>
<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
using the Compound File format (default in 1.4 and greater) these files (except
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,
although this is not required.</p>
<p>As of version 2.1 (lock-less commits), file names are never re-used (there
is one exception, "segments.gen", see below). That is, when any file is saved
to the Directory it is given a never before used filename. This is achieved
using a simple generations approach. For example, the first segments file is
segments_1, then segments_2, etc. The generation is a sequential long integer
represented in alpha-numeric (base 36) form.</p>
</div>
<a name="file-names" id="file-names"></a>
<h2>Summary of File Extensions</h2>
<div>
<p>The following table summarizes the names and extensions of the files in
Lucene:</p>
<table cellspacing="1" cellpadding="4">
<tr>
<th>Name</th>
<th>Extension</th>
<th>Brief Description</th>
</tr>
<tr>
<td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
<td>segments.gen, segments_N</td>
<td>Stores information about a commit point</td>
</tr>
<tr>
<td><a href="#Lock_File">Lock File</a></td>
<td>write.lock</td>
<td>The Write lock prevents multiple IndexWriters from writing to the same
file.</td>
</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>.cfs, .cfe</td>
<td>An optional "virtual" file consisting of all the other index files for
systems that frequently run out of file handles.</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene46.Lucene46FieldInfosFormat Fields}</td>
<td>.fnm</td>
<td>Stores information about the fields</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Index}</td>
<td>.fdx</td>
<td>Contains pointers to field data</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Data}</td>
<td>.fdt</td>
<td>The stored fields for documents</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary}</td>
<td>.tim</td>
<td>The term dictionary, stores term info</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index}</td>
<td>.tip</td>
<td>The index into the Term Dictionary</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Frequencies}</td>
<td>.doc</td>
<td>Contains the list of docs which contain each term along with frequency</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Positions}</td>
<td>.pos</td>
<td>Stores position information about where a term occurs in the index</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Payloads}</td>
<td>.pay</td>
<td>Stores additional per-position metadata information such as character offsets and user payloads</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42NormsFormat Norms}</td>
<td>.nvd, .nvm</td>
<td>Encodes length and boost factors for docs and fields</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat Per-Document Values}</td>
<td>.dvd, .dvm</td>
<td>Encodes additional scoring factors or other per-document information.</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Index}</td>
<td>.tvx</td>
<td>Stores offset into the document data file</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Documents}</td>
<td>.tvd</td>
<td>Contains information about each document that has term vectors</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Fields}</td>
<td>.tvf</td>
<td>The field level info about term vectors</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted Documents}</td>
<td>.del</td>
<td>Info about what files are deleted</td>
</tr>
</table>
</div>
<a name="Lock_File" id="Lock_File"></a>
<h2>Lock File</h2>
The write lock, which is stored in the index directory by default, is named
"write.lock". If the lock directory is different from the index directory then
the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
derived from the full path to the index directory. When this file is present, a
writer is currently modifying the index (adding or removing documents). This
lock file ensures that only one writer is modifying the index at a time.</p>
<a name="History"></a>
<h2>History</h2>
<p>Compatibility notes are provided in this document, describing how file
formats have changed from prior versions:</p>
<ul>
<li>In version 2.1, the file format was changed to allow lock-less commits (ie,
no more commit lock). The change is fully backwards compatible: you can open a
pre-2.1 index for searching or adding/deleting of docs. When the new segments
file is saved (committed), it will be written in the new file format (meaning
no specific "upgrade" process is needed). But note that once a commit has
occurred, pre-2.1 Lucene will not be able to read the index.</li>
<li>In version 2.3, the file format was changed to allow segments to share a
single set of doc store (vectors &amp; stored fields) files. This allows for
faster indexing in certain cases. The change is fully backwards compatible (in
the same way as the lock-less commits change in 2.1).</li>
<li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
LUCENE-510</a> for details.</li>
<li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
may be passed to IndexWriter's commit methods (and later retrieved), which is
recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
LUCENE-1382</a> for details. Also,
diagnostics were added to each segment written recording details about why it
was written (due to flush, merge; which OS/JRE was used; etc.). See issue
<a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
<li>In version 3.0, compressed fields are no longer written to the index (they
can still be read, but on merge the new segment will write them, uncompressed).
See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
for details.</li>
<li>In version 3.1, segments records the code version that created them. See
<a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
Additionally segments track explicitly whether or not they have term vectors.
See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
for details.</li>
<li>In version 3.2, numeric fields are written as natively to stored fields
file, previously they were stored in text format only.</li>
<li>In version 3.4, fields can omit position data while still indexing term
frequencies.</li>
<li>In version 4.0, the format of the inverted index became extensible via
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
({@code DocValues}) was introduced. Normalization factors need no longer be a
single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
Terms need not be unicode strings, they can be any byte sequence. Term offsets
can optionally be indexed into the postings lists. Payloads can be stored in the
term vectors.</li>
<li>In version 4.1, the format of the postings list changed to use either
of FOR compression or variable-byte encoding, depending upon the frequency
of the term. Terms appearing only once were changed to inline directly into
the term dictionary. Stored fields are compressed by default. </li>
<li>In version 4.2, term vectors are compressed by default. DocValues has
a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
on multi-valued fields.</li>
<li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
<li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
allow updating NumericDocValues fields.</li>
</ul>
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>
<div>
<p>Lucene uses a Java <code>int</code> to refer to
document numbers, and the index file format uses an <code>Int32</code>
on-disk to store document numbers. This is a limitation
of both the index file format and the current implementation. Eventually these
should be replaced with either <code>UInt64</code> values, or
better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
</div>
</body>
</html>

View File

@ -61,13 +61,13 @@ name of your codec.
If you just want to customise the {@link org.apache.lucene.codecs.PostingsFormat}, or use different postings
formats for different fields, then you can register your custom postings format in the same way (in
META-INF/services/org.apache.lucene.codecs.PostingsFormat), and then extend the default
{@link org.apache.lucene.codecs.lucene45.Lucene45Codec} and override
{@link org.apache.lucene.codecs.lucene45.Lucene45Codec#getPostingsFormatForField(String)} to return your custom
{@link org.apache.lucene.codecs.lucene46.Lucene46Codec} and override
{@link org.apache.lucene.codecs.lucene46.Lucene46Codec#getPostingsFormatForField(String)} to return your custom
postings format.
</p>
<p>
Similarly, if you just want to customise the {@link org.apache.lucene.codecs.DocValuesFormat} per-field, have
a look at {@link org.apache.lucene.codecs.lucene45.Lucene45Codec#getDocValuesFormatForField(String)}.
a look at {@link org.apache.lucene.codecs.lucene46.Lucene46Codec#getDocValuesFormatForField(String)}.
</p>
</body>
</html>

View File

@ -122,12 +122,15 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
}
private DocValuesConsumer getInstance(FieldInfo field) throws IOException {
final DocValuesFormat format;
if (segmentWriteState.isFieldUpdate) {
DocValuesFormat format = null;
if (field.getDocValuesGen() != -1) {
final String formatName = field.getAttribute(PER_FIELD_FORMAT_KEY);
assert formatName != null : "invalid null FORMAT_KEY for field=\"" + field.name + "\" (field updates)";
// this means the field never existed in that segment, yet is applied updates
if (formatName != null) {
format = DocValuesFormat.forName(formatName);
} else {
}
}
if (format == null) {
format = getDocValuesFormatForField(field.name);
}
if (format == null) {
@ -136,19 +139,25 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
final String formatName = format.getName();
String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
assert segmentWriteState.isFieldUpdate || previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
assert field.getDocValuesGen() != -1 || previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
Integer suffix;
Integer suffix = null;
ConsumerAndSuffix consumer = formats.get(format);
if (consumer == null) {
// First time we are seeing this format; create a new instance
if (segmentWriteState.isFieldUpdate) {
if (field.getDocValuesGen() != -1) {
final String suffixAtt = field.getAttribute(PER_FIELD_SUFFIX_KEY);
assert suffixAtt != null : "invalid numm SUFFIX_KEY for field=\"" + field.name + "\" (field updates)";
// even when dvGen is != -1, it can still be a new field, that never
// existed in the segment, and therefore doesn't have the recorded
// attributes yet.
if (suffixAtt != null) {
suffix = Integer.valueOf(suffixAtt);
} else {
}
}
if (suffix == null) {
// bump the suffix
suffix = suffixes.get(formatName);
if (suffix == null) {
@ -172,7 +181,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
}
previousValue = field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
assert segmentWriteState.isFieldUpdate || previousValue == null : "suffix=" + Integer.toString(suffix) + " prevValue=" + previousValue;
assert field.getDocValuesGen() != -1 || previousValue == null : "suffix=" + Integer.toString(suffix) + " prevValue=" + previousValue;
// TODO: we should only provide the "slice" of FIS
// that this DVF actually sees ...

View File

@ -85,7 +85,7 @@ final class DocFieldProcessor extends DocConsumer {
// FreqProxTermsWriter does this with
// FieldInfo.storePayload.
FieldInfosWriter infosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
infosWriter.write(state.directory, state.segmentInfo.name, state.fieldInfos, IOContext.DEFAULT);
infosWriter.write(state.directory, state.segmentInfo.name, "", state.fieldInfos, IOContext.DEFAULT);
}
@Override

View File

@ -47,6 +47,8 @@ public final class FieldInfo {
private Map<String,String> attributes;
private long dvGen = -1; // the DocValues generation of this field
/**
* Controls how much information is stored in the postings lists.
* @lucene.experimental
@ -117,8 +119,9 @@ public final class FieldInfo {
*
* @lucene.experimental
*/
public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector,
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, Map<String,String> attributes) {
public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector, boolean omitNorms,
boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType,
Map<String,String> attributes) {
this.name = name;
this.indexed = indexed;
this.number = number;
@ -223,6 +226,19 @@ public final class FieldInfo {
return docValueType;
}
/** Sets the docValues generation of this field. */
public void setDocValuesGen(long dvGen) {
this.dvGen = dvGen;
}
/**
* Returns the docValues generation of this field, or -1 if no docValues
* updates exist for it.
*/
public long getDocValuesGen() {
return dvGen;
}
/**
* Returns {@link DocValuesType} of the norm. this may be null if the field has no norms.
*/

View File

@ -532,6 +532,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
final ReadersAndLiveDocs rld = readerMap.get(info);
if (rld != null) {
assert rld.info == info;
boolean hasFieldUpdates = rld.hasFieldUpdates(); // only reopen reader if there were field udpates
if (rld.writeLiveDocs(directory)) {
// Make sure we only write del docs and updates for a live segment:
assert infoIsLive(info);
@ -540,12 +541,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
// created new _X_N.del and field updates files.
deleter.checkpoint(segmentInfos, false);
// we wrote liveDocs and field updates, reopen the reader
// we wrote field updates, reopen the reader
if (hasFieldUpdates) {
rld.reopenReader(IOContext.READ);
}
}
}
}
}
/**
* Obtain a ReadersAndLiveDocs instance from the
@ -783,27 +786,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
}
}
private FieldInfos getFieldInfos(SegmentInfo info) throws IOException {
Directory cfsDir = null;
try {
if (info.getUseCompoundFile()) {
cfsDir = new CompoundFileDirectory(info.dir,
IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
IOContext.READONCE,
false);
} else {
cfsDir = info.dir;
}
return info.getCodec().fieldInfosFormat().getFieldInfosReader().read(cfsDir,
info.name,
IOContext.READONCE);
} finally {
if (info.getUseCompoundFile() && cfsDir != null) {
cfsDir.close();
}
}
}
/**
* Loads or returns the already loaded the global field number map for this {@link SegmentInfos}.
* If this {@link SegmentInfos} has no global field number map the returned instance is empty
@ -812,7 +794,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
final FieldNumbers map = new FieldNumbers();
for(SegmentInfoPerCommit info : segmentInfos) {
for(FieldInfo fi : getFieldInfos(info.info)) {
for(FieldInfo fi : SegmentReader.readFieldInfos(info)) {
map.addOrGet(fi.name, fi.number, fi.getDocValuesType());
}
}
@ -1547,13 +1529,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
* that already exist in the index, not add new fields through this method.
*
* <p>
* <b>NOTE:</b> it is currently not allowed to update the value of documents
* in a segment where the field does not exist (even though it may exist in
* other segments). If you try that, you will hit an
* {@link UnsupportedOperationException} when the segment is later flushed
* (following an NRT reader reopen, commit, forceMerge etc.).
*
* <p>
* <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
* close the writer. See <a href="#OOME">above</a> for details.
* </p>
@ -1569,7 +1544,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
* @throws IOException
* if there is a low-level IO error
*/
// TODO (DVU_FIELDINFOS_GEN) remove the paragraph on updating segments without the field not allowed
public void updateNumericDocValue(Term term, String field, Long value) throws IOException {
ensureOpen();
if (!globalFieldNumberMap.contains(field, DocValuesType.NUMERIC)) {
@ -2431,7 +2405,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
IOContext context = new IOContext(new MergeInfo(info.info.getDocCount(), info.sizeInBytes(), true, -1));
for(FieldInfo fi : getFieldInfos(info.info)) {
for(FieldInfo fi : SegmentReader.readFieldInfos(info)) {
globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getDocValuesType());
}
infos.add(copySegmentAsIs(info, newSegName, context));
@ -2632,7 +2606,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
// note: we don't really need this fis (its copied), but we load it up
// so we don't pass a null value to the si writer
FieldInfos fis = getFieldInfos(info.info);
FieldInfos fis = SegmentReader.readFieldInfos(info);
final Map<String,String> attributes;
// copy the attributes map, we might modify it below.
@ -2648,7 +2622,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
info.info.getUseCompoundFile(),
info.info.getCodec(), info.info.getDiagnostics(), attributes);
SegmentInfoPerCommit newInfoPerCommit = new SegmentInfoPerCommit(newInfo, info.getDelCount(), info.getDelGen(), info.getDocValuesGen());
SegmentInfoPerCommit newInfoPerCommit = new SegmentInfoPerCommit(newInfo, info.getDelCount(), info.getDelGen(), info.getFieldInfosGen());
Set<String> segFiles = new HashSet<String>();

View File

@ -29,7 +29,7 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.TrackingDirectoryWrapper;
@ -144,9 +144,6 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
try {
reader.decRef();
reader = newReader;
if (liveDocs == null) {
liveDocs = reader.getLiveDocs();
}
reopened = true;
} finally {
if (!reopened) {
@ -328,7 +325,7 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
// discard them on the sub-readers:
pendingDeleteCount = 0;
numericUpdates.clear();
mergingUpdates.clear();
dropMergingUpdates();
}
// Commit live docs (writes new _X_N.del files) and field updates (writes new
@ -344,9 +341,7 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
}
// We have new deletes or updates
if (pendingDeleteCount > 0) {
assert liveDocs.length() == info.info.getDocCount();
}
assert pendingDeleteCount == 0 || liveDocs.length() == info.info.getDocCount();
// Do this so we can delete any created files on
// exception; this saves all codecs from having to do
@ -370,11 +365,11 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
// if (this.reader == null) System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: newSR " + info);
final SegmentReader reader = this.reader == null ? new SegmentReader(info, IOContext.READONCE) : this.reader;
try {
// clone FieldInfos so that we can update their numericUpdatesGen
// separately from the reader's infos and write them to a new
// fieldInfos_gen file
// clone FieldInfos so that we can update their dvGen separately from
// the reader's infos and write them to a new fieldInfos_gen file
FieldInfos.Builder builder = new FieldInfos.Builder(writer.globalFieldNumberMap);
// cannot use builder.add(reader.getFieldInfos()) because it does not clone FI.attributes
// cannot use builder.add(reader.getFieldInfos()) because it does not
// clone FI.attributes as well FI.dvGen
for (FieldInfo fi : reader.getFieldInfos()) {
FieldInfo clone = builder.add(fi);
// copy the stuff FieldInfos.Builder doesn't copy
@ -383,16 +378,17 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
clone.putAttribute(e.getKey(), e.getValue());
}
}
clone.setDocValuesGen(fi.getDocValuesGen());
}
// create new fields or update existing ones to have NumericDV type
// for (String f : numericUpdates.keySet()) {
// builder.addOrUpdate(f, NumericDocValuesField.TYPE);
// }
for (String f : numericUpdates.keySet()) {
builder.addOrUpdate(f, NumericDocValuesField.TYPE);
}
final FieldInfos fieldInfos = builder.finish();
final long nextDocValuesGen = info.getNextDocValuesGen();
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, IOContext.DEFAULT, segmentSuffix, true);
final long nextFieldInfosGen = info.getNextFieldInfosGen();
final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, IOContext.DEFAULT, segmentSuffix);
final DocValuesFormat docValuesFormat = codec.docValuesFormat();
final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
boolean fieldsConsumerSuccess = false;
@ -403,13 +399,9 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
final Map<Integer,Long> updates = e.getValue();
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
if (fieldInfo == null || fieldInfo.getDocValuesType() != DocValuesType.NUMERIC) {
throw new UnsupportedOperationException(
"cannot update docvalues in a segment with no docvalues field: segment=" + info + ", field=" + field);
}
// assert fieldInfo != null;
assert fieldInfo != null;
info.setDocValuesGen(fieldInfo.number, nextDocValuesGen);
fieldInfo.setDocValuesGen(nextFieldInfosGen);
// write the numeric updates to a new gen'd docvalues file
fieldsConsumer.addNumericField(fieldInfo, new Iterable<Number>() {
@ -451,6 +443,8 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
}
});
}
codec.fieldInfosFormat().getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, IOContext.DEFAULT);
fieldsConsumerSuccess = true;
} finally {
if (fieldsConsumerSuccess) {
@ -478,7 +472,7 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
// Advance only the nextWriteDocValuesGen so that a 2nd
// attempt to write will write to a new file
if (hasFieldUpdates) {
info.advanceNextWriteDocValuesGen();
info.advanceNextWriteFieldInfosGen();
}
// Delete any partially created file(s):
@ -502,7 +496,7 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
}
if (hasFieldUpdates) {
info.advanceDocValuesGen();
info.advanceFieldInfosGen();
// copy all the updates to mergingUpdates, so they can later be applied to the merged segment
if (isMerging) {
copyUpdatesToMerging();

View File

@ -50,7 +50,6 @@ final class SegmentCoreReaders {
// SegmentReaders:
private final AtomicInteger ref = new AtomicInteger(1);
final FieldInfos fieldInfos;
final FieldsProducer fields;
final DocValuesProducer normsProducer;
@ -103,10 +102,10 @@ final class SegmentCoreReaders {
cfsDir = dir;
}
fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE);
final FieldInfos fieldInfos = owner.fieldInfos;
final PostingsFormat format = codec.postingsFormat();
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context);
final PostingsFormat format = codec.postingsFormat();
// Ask codec for its Fields
fields = format.fieldsProducer(segmentReadState);
assert fields != null;
@ -151,24 +150,15 @@ final class SegmentCoreReaders {
ref.incrementAndGet();
}
NumericDocValues getNormValues(String field) throws IOException {
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {
// Field does not exist
return null;
}
if (!fi.hasNorms()) {
return null;
}
NumericDocValues getNormValues(FieldInfo fi) throws IOException {
assert normsProducer != null;
Map<String,Object> normFields = normsLocal.get();
NumericDocValues norms = (NumericDocValues) normFields.get(field);
NumericDocValues norms = (NumericDocValues) normFields.get(fi.name);
if (norms == null) {
norms = normsProducer.getNumeric(fi);
normFields.put(field, norms);
normFields.put(fi.name, norms);
}
return norms;

View File

@ -19,9 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.store.Directory;
@ -46,16 +44,12 @@ public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo
// attempt to write:
private long nextWriteDelGen;
// holds field.number to docValuesGen mapping
// TODO (DVU_FIELDINFOS_GEN) once we gen FieldInfos, get rid of this; every FieldInfo will record its dvGen
private final Map<Integer,Long> fieldDocValuesGens = new HashMap<Integer,Long>();
// Generation number of the FieldInfos (-1 if there are no updates)
private long fieldInfosGen;
// Generation number of the docValues (-1 if there are no field updates)
private long docValuesGen;
// Normally 1 + docValuesGen, unless an exception was hit on last attempt to
// Normally 1 + fieldInfosGen, unless an exception was hit on last attempt to
// write
private long nextWriteDocValuesGen;
private long nextWriteFieldInfosGen;
// Tracks the files with field updates
private Set<String> updatesFiles = new HashSet<String>();
@ -71,10 +65,10 @@ public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo
* number of deleted documents in this segment
* @param delGen
* deletion generation number (used to name deletion files)
* @param docValuesGen
* doc-values generation number (used to name docvalues files)
* @param fieldInfosGen
* FieldInfos generation number (used to name field-infos files)
**/
public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen, long docValuesGen) {
public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen, long fieldInfosGen) {
this.info = info;
this.delCount = delCount;
this.delGen = delGen;
@ -84,11 +78,11 @@ public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo
nextWriteDelGen = delGen+1;
}
this.docValuesGen = docValuesGen;
if (docValuesGen == -1) {
nextWriteDocValuesGen = 1;
this.fieldInfosGen = fieldInfosGen;
if (fieldInfosGen == -1) {
nextWriteFieldInfosGen = 1;
} else {
nextWriteDocValuesGen = docValuesGen + 1;
nextWriteFieldInfosGen = fieldInfosGen + 1;
}
}
@ -116,19 +110,19 @@ public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo
nextWriteDelGen++;
}
/** Called when we succeed in writing docvalues updates */
void advanceDocValuesGen() {
docValuesGen = nextWriteDocValuesGen;
nextWriteDocValuesGen = docValuesGen + 1;
/** Called when we succeed in writing a new FieldInfos generation. */
void advanceFieldInfosGen() {
fieldInfosGen = nextWriteFieldInfosGen;
nextWriteFieldInfosGen = fieldInfosGen + 1;
sizeInBytes = -1;
}
/**
* Called if there was an exception while writing docvalues updates, so that
* we don't try to write to the same file more than once.
* Called if there was an exception while writing a new generation of
* FieldInfos, so that we don't try to write to the same file more than once.
*/
void advanceNextWriteDocValuesGen() {
nextWriteDocValuesGen++;
void advanceNextWriteFieldInfosGen() {
nextWriteFieldInfosGen++;
}
/** Returns total size in bytes of all files for this
@ -183,43 +177,20 @@ public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo
/** Returns true if there are any field updates for the segment in this commit. */
public boolean hasFieldUpdates() {
return docValuesGen != -1;
return fieldInfosGen != -1;
}
/** Returns the next available generation number of the docvalues files. */
public long getNextDocValuesGen() {
return nextWriteDocValuesGen;
}
/**
* Returns the docvalues generation of this field, or -1 if there are
* no updates to it.
*/
public long getDocValuesGen(int fieldNumber) {
Long gen = fieldDocValuesGens.get(fieldNumber);
return gen == null ? -1 : gen.longValue();
}
/** Sets the docvalues generation for this field. */
public void setDocValuesGen(int fieldNumber, long gen) {
fieldDocValuesGens.put(fieldNumber, gen);
}
/**
* Returns a mapping from a field number to its DV generation.
*
* @see #getDocValuesGen(int)
*/
public Map<Integer,Long> getFieldDocValuesGens() {
return fieldDocValuesGens;
/** Returns the next available generation number of the FieldInfos files. */
public long getNextFieldInfosGen() {
return nextWriteFieldInfosGen;
}
/**
* Returns the generation number of the field infos file or -1 if there are no
* field updates yet.
*/
public long getDocValuesGen() {
return docValuesGen;
public long getFieldInfosGen() {
return fieldInfosGen;
}
/**
@ -261,25 +232,24 @@ public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo
if (delGen != -1) {
s += ":delGen=" + delGen;
}
if (docValuesGen != -1) {
s += ":docValuesGen=" + docValuesGen;
if (fieldInfosGen != -1) {
s += ":fieldInfosGen=" + fieldInfosGen;
}
return s;
}
@Override
public SegmentInfoPerCommit clone() {
SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen, docValuesGen);
SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen);
// Not clear that we need to carry over nextWriteDelGen
// (i.e. do we ever clone after a failed write and
// before the next successful write?), but just do it to
// be safe:
other.nextWriteDelGen = nextWriteDelGen;
other.nextWriteDocValuesGen = nextWriteDocValuesGen;
other.nextWriteFieldInfosGen = nextWriteFieldInfosGen;
other.updatesFiles.addAll(updatesFiles);
other.fieldDocValuesGens.putAll(fieldDocValuesGens);
return other;
}
}

View File

@ -28,11 +28,11 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.ChecksumIndexOutput;
@ -70,7 +70,7 @@ import org.apache.lucene.util.IOUtils;
* <ul>
* <li><tt>segments.gen</tt>: GenHeader, Generation, Generation
* <li><tt>segments_N</tt>: Header, Version, NameCounter, SegCount,
* &lt;SegName, SegCodec, DelGen, DeletionCount&gt;<sup>SegCount</sup>,
* &lt;SegName, SegCodec, DelGen, DeletionCount, FieldInfosGen, UpdatesFiles&gt;<sup>SegCount</sup>,
* CommitUserData, Checksum
* </ul>
* </p>
@ -79,9 +79,10 @@ import org.apache.lucene.util.IOUtils;
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>GenHeader, NameCounter, SegCount, DeletionCount --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>Generation, Version, DelGen, Checksum --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>Generation, Version, DelGen, Checksum, FieldInfosGen --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>SegName, SegCodec --&gt; {@link DataOutput#writeString String}</li>
* <li>CommitUserData --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
* <li>UpdatesFiles --&gt; {@link DataOutput#writeStringSet(Set) Set&lt;String&gt;}</li>
* </ul>
* </p>
* Field Descriptions:
@ -104,6 +105,10 @@ import org.apache.lucene.util.IOUtils;
* <li>CommitUserData stores an optional user-supplied opaque
* Map&lt;String,String&gt; that was passed to
* {@link IndexWriter#setCommitData(java.util.Map)}.</li>
* <li>FieldInfosGen is the generation count of the fieldInfos file. If this is -1,
* there are no updates to the fieldInfos in that segment. Anything above zero
* means there are updates to fieldInfos stored by {@link FieldInfosFormat}.</li>
* <li>UpdatesFiles stores the list of files that were updated in that segment.</li>
* </ul>
* </p>
*
@ -111,11 +116,11 @@ import org.apache.lucene.util.IOUtils;
*/
public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCommit> {
/** The file format version for the segments_N codec header, up to 4.4. */
/** The file format version for the segments_N codec header, up to 4.5. */
public static final int VERSION_40 = 0;
/** The file format version for the segments_N codec header, since 4.5+. */
public static final int VERSION_45 = 1;
/** The file format version for the segments_N codec header, since 4.6+. */
public static final int VERSION_46 = 1;
/** Used for the segments.gen file only!
* Whenever you add a new format, make it 1 smaller (negative version logic)! */
@ -320,7 +325,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
}
// 4.0+
int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_45);
int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_46);
version = input.readLong();
counter = input.readInt();
int numSegments = input.readInt();
@ -338,16 +343,12 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
if (delCount < 0 || delCount > info.getDocCount()) {
throw new CorruptIndexException("invalid deletion count: " + delCount + " (resource: " + input + ")");
}
long docValuesGen = -1;
if (format >= VERSION_45) {
docValuesGen = input.readLong();
}
SegmentInfoPerCommit siPerCommit = new SegmentInfoPerCommit(info, delCount, delGen, docValuesGen);
if (format >= VERSION_45) {
int numUpdates = input.readInt();
for (int i = 0; i < numUpdates; i++) {
siPerCommit.setDocValuesGen(input.readInt(), input.readLong());
long fieldInfosGen = -1;
if (format >= VERSION_46) {
fieldInfosGen = input.readLong();
}
SegmentInfoPerCommit siPerCommit = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen);
if (format >= VERSION_46) {
siPerCommit.addUpdatesFiles(input.readStringSet());
}
add(siPerCommit);
@ -408,7 +409,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
try {
segnOutput = new ChecksumIndexOutput(directory.createOutput(segmentFileName, IOContext.DEFAULT));
CodecUtil.writeHeader(segnOutput, "segments", VERSION_45);
CodecUtil.writeHeader(segnOutput, "segments", VERSION_46);
segnOutput.writeLong(version);
segnOutput.writeInt(counter); // write counter
segnOutput.writeInt(size()); // write infos
@ -418,13 +419,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
segnOutput.writeString(si.getCodec().getName());
segnOutput.writeLong(siPerCommit.getDelGen());
segnOutput.writeInt(siPerCommit.getDelCount());
segnOutput.writeLong(siPerCommit.getDocValuesGen());
Map<Integer,Long> docValuesUpdatesGen = siPerCommit.getFieldDocValuesGens();
segnOutput.writeInt(docValuesUpdatesGen.size());
for (Entry<Integer,Long> e : docValuesUpdatesGen.entrySet()) {
segnOutput.writeInt(e.getKey());
segnOutput.writeLong(e.getValue());
}
segnOutput.writeLong(siPerCommit.getFieldInfosGen());
segnOutput.writeStringSet(siPerCommit.getUpdatesFiles());
assert si.dir == directory;

View File

@ -142,7 +142,7 @@ final class SegmentMerger {
// write the merged infos
FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
fieldInfosWriter.write(directory, mergeState.segmentInfo.name, mergeState.fieldInfos, context);
fieldInfosWriter.write(directory, mergeState.segmentInfo.name, "", mergeState.fieldInfos, context);
return mergeState;
}

View File

@ -31,6 +31,7 @@ import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
@ -73,6 +74,8 @@ public final class SegmentReader extends AtomicReader {
final Map<String,DocValuesProducer> dvProducers = new HashMap<String,DocValuesProducer>();
final Map<Long,RefCount<DocValuesProducer>> genDVProducers = new HashMap<Long,RefCount<DocValuesProducer>>();
final FieldInfos fieldInfos;
/**
* Constructs a new SegmentReader with a new core.
* @throws CorruptIndexException if the index is corrupt
@ -81,6 +84,13 @@ public final class SegmentReader extends AtomicReader {
// TODO: why is this public?
public SegmentReader(SegmentInfoPerCommit si, IOContext context) throws IOException {
this.si = si;
// TODO if the segment uses CFS, we may open the CFS file twice: once for
// reading the FieldInfos (if they are not gen'd) and second time by
// SegmentCoreReaders. We can open the CFS here and pass to SCR, but then it
// results in less readable code (resource not closed where it was opened).
// Best if we could somehow read FieldInfos in SCR but not keep it there, but
// constructors don't allow returning two things...
fieldInfos = readFieldInfos(si);
core = new SegmentCoreReaders(this, si.info.dir, si, context);
boolean success = false;
@ -95,7 +105,7 @@ public final class SegmentReader extends AtomicReader {
}
numDocs = si.info.getDocCount() - si.getDelCount();
if (core.fieldInfos.hasDocValues()) {
if (fieldInfos.hasDocValues()) {
final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
final DocValuesFormat dvFormat = codec.docValuesFormat();
// initialize the per generation numericDVProducers and put the correct
@ -157,8 +167,14 @@ public final class SegmentReader extends AtomicReader {
// increment refCount of DocValuesProducers that are used by this reader
boolean success = false;
try {
if (core.fieldInfos.hasDocValues()) {
final Codec codec = si.info.getCodec();
if (si.getFieldInfosGen() == -1) {
fieldInfos = sr.fieldInfos;
} else {
fieldInfos = readFieldInfos(si);
}
if (fieldInfos.hasDocValues()) {
final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
final DocValuesFormat dvFormat = codec.docValuesFormat();
@ -196,14 +212,45 @@ public final class SegmentReader extends AtomicReader {
}
}
/**
* Reads the most recent {@link FieldInfos} of the given segment info.
*
* @lucene.internal
*/
static FieldInfos readFieldInfos(SegmentInfoPerCommit info) throws IOException {
final Directory dir;
final boolean closeDir;
if (info.getFieldInfosGen() == -1 && info.info.getUseCompoundFile()) {
// no fieldInfos gen and segment uses a compound file
dir = new CompoundFileDirectory(info.info.dir,
IndexFileNames.segmentFileName(info.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
IOContext.READONCE,
false);
closeDir = true;
} else {
// gen'd FIS are read outside CFS, or the segment doesn't use a compound file
dir = info.info.dir;
closeDir = false;
}
try {
final String segmentSuffix = info.getFieldInfosGen() == -1 ? "" : Long.toString(info.getFieldInfosGen(), Character.MAX_RADIX);
return info.info.getCodec().fieldInfosFormat().getFieldInfosReader().read(dir, info.info.name, segmentSuffix, IOContext.READONCE);
} finally {
if (closeDir) {
dir.close();
}
}
}
// returns a gen->List<FieldInfo> mapping. Fields without DV updates have gen=-1
private Map<Long,List<FieldInfo>> getGenInfos(SegmentInfoPerCommit si) {
final Map<Long,List<FieldInfo>> genInfos = new HashMap<Long,List<FieldInfo>>();
for (FieldInfo fi : core.fieldInfos) {
for (FieldInfo fi : fieldInfos) {
if (fi.getDocValuesType() == null) {
continue;
}
long gen = si.getDocValuesGen(fi.number);
long gen = fi.getDocValuesGen();
List<FieldInfo> infos = genInfos.get(gen);
if (infos == null) {
infos = new ArrayList<FieldInfo>();
@ -267,7 +314,7 @@ public final class SegmentReader extends AtomicReader {
@Override
public FieldInfos getFieldInfos() {
ensureOpen();
return core.fieldInfos;
return fieldInfos;
}
/** Expert: retrieve thread-private {@link
@ -372,7 +419,7 @@ public final class SegmentReader extends AtomicReader {
// null if the field does not exist, or not indexed as the requested
// DovDocValuesType.
private FieldInfo getDVField(String field, DocValuesType type) {
FieldInfo fi = core.fieldInfos.fieldInfo(field);
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {
// Field does not exist
return null;
@ -414,7 +461,7 @@ public final class SegmentReader extends AtomicReader {
@Override
public Bits getDocsWithField(String field) throws IOException {
ensureOpen();
FieldInfo fi = core.fieldInfos.fieldInfo(field);
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {
// Field does not exist
return null;
@ -507,7 +554,12 @@ public final class SegmentReader extends AtomicReader {
@Override
public NumericDocValues getNormValues(String field) throws IOException {
ensureOpen();
return core.getNormValues(field);
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null || !fi.hasNorms()) {
// Field does not exist or does not index norms
return null;
}
return core.getNormValues(fi);
}
/**

View File

@ -71,24 +71,20 @@ public class SegmentWriteState {
* to {@link Directory#createOutput(String,IOContext)}. */
public final IOContext context;
/** True is this instance represents a field update. */
public final boolean isFieldUpdate; // TODO (DVU_FIELDINFOS_GEN) once we gen FieldInfos, get rid of this
/** Sole constructor. */
public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
BufferedDeletes segDeletes, IOContext context) {
this(infoStream, directory, segmentInfo, fieldInfos, segDeletes, context, "", false);
this(infoStream, directory, segmentInfo, fieldInfos, segDeletes, context, "");
}
/**
* Constructor which takes segment suffix and isFieldUpdate in addition to the
* other parameters.
* Constructor which takes segment suffix.
*
* @see #SegmentWriteState(InfoStream, Directory, SegmentInfo, FieldInfos,
* BufferedDeletes, IOContext)
*/
public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
BufferedDeletes segDeletes, IOContext context, String segmentSuffix, boolean isFieldUpdate) {
BufferedDeletes segDeletes, IOContext context, String segmentSuffix) {
this.infoStream = infoStream;
this.segDeletes = segDeletes;
this.directory = directory;
@ -96,7 +92,6 @@ public class SegmentWriteState {
this.fieldInfos = fieldInfos;
this.segmentSuffix = segmentSuffix;
this.context = context;
this.isFieldUpdate = isFieldUpdate;
}
/** Create a shallow copy of {@link SegmentWriteState} with a new segment suffix. */
@ -109,6 +104,5 @@ public class SegmentWriteState {
this.segmentSuffix = segmentSuffix;
segDeletes = state.segDeletes;
delCountOnFlush = state.delCountOnFlush;
isFieldUpdate = state.isFieldUpdate;
}
}

View File

@ -163,7 +163,7 @@ final class StandardDirectoryReader extends DirectoryReader {
newReaders[i] = newReader;
} else {
if (newReaders[i].getSegmentInfo().getDelGen() == infos.info(i).getDelGen()
&& newReaders[i].getSegmentInfo().getDocValuesGen() == infos.info(i).getDocValuesGen()) {
&& newReaders[i].getSegmentInfo().getFieldInfosGen() == infos.info(i).getFieldInfosGen()) {
// No change; this reader will be shared between
// the old and the new one, so we must incRef
// it:

View File

@ -17,3 +17,4 @@ org.apache.lucene.codecs.lucene40.Lucene40Codec
org.apache.lucene.codecs.lucene41.Lucene41Codec
org.apache.lucene.codecs.lucene42.Lucene42Codec
org.apache.lucene.codecs.lucene45.Lucene45Codec
org.apache.lucene.codecs.lucene46.Lucene46Codec

View File

@ -19,7 +19,7 @@ package org.apache.lucene;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.DirectoryReader;
@ -37,7 +37,7 @@ import org.apache.lucene.util.LuceneTestCase;
public class TestExternalCodecs extends LuceneTestCase {
private static final class CustomPerFieldCodec extends Lucene45Codec {
private static final class CustomPerFieldCodec extends Lucene46Codec {
private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");

View File

@ -25,7 +25,7 @@ import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -81,7 +81,7 @@ public class TestPerFieldDocValuesFormat extends BaseDocValuesFormatTestCase {
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
final DocValuesFormat fast = DocValuesFormat.forName("Lucene45");
final DocValuesFormat slow = DocValuesFormat.forName("SimpleText");
iwc.setCodec(new Lucene45Codec() {
iwc.setCodec(new Lucene46Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
if ("dv1".equals(field)) {

View File

@ -21,8 +21,8 @@ import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
@ -34,10 +34,10 @@ import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.LogDocMergePolicy;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
@ -200,7 +200,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
public static class MockCodec extends Lucene45Codec {
public static class MockCodec extends Lucene46Codec {
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
final PostingsFormat mockSep = new MockSepPostingsFormat();
@ -217,7 +217,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
}
public static class MockCodec2 extends Lucene45Codec {
public static class MockCodec2 extends Lucene46Codec {
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
@ -268,7 +268,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
public void testSameCodecDifferentInstance() throws Exception {
Codec codec = new Lucene45Codec() {
Codec codec = new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
if ("id".equals(field)) {
@ -284,7 +284,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
public void testSameCodecDifferentParams() throws Exception {
Codec codec = new Lucene45Codec() {
Codec codec = new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
if ("id".equals(field)) {

View File

@ -28,7 +28,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -43,7 +43,6 @@ import org.apache.lucene.store.BaseDirectoryWrapper;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
@ -1060,7 +1059,7 @@ public class TestAddIndexes extends LuceneTestCase {
aux2.close();
}
private static final class CustomPerFieldCodec extends Lucene45Codec {
private static final class CustomPerFieldCodec extends Lucene46Codec {
private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final PostingsFormat mockSepFormat = PostingsFormat.forName("MockSep");
@ -1111,7 +1110,7 @@ public class TestAddIndexes extends LuceneTestCase {
private static final class UnRegisteredCodec extends FilterCodec {
public UnRegisteredCodec() {
super("NotRegistered", new Lucene45Codec());
super("NotRegistered", new Lucene46Codec());
}
}

View File

@ -21,12 +21,9 @@ import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@ -41,7 +38,7 @@ public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
public void test() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setCodec(new Lucene45Codec());
conf.setCodec(new Lucene46Codec());
// riw should sometimes create docvalues fields, etc
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
Document doc = new Document();

View File

@ -37,8 +37,8 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.FloatDocValuesField;
import org.apache.lucene.document.IntField;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.LongField;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
@ -54,19 +54,18 @@ import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.BaseDirectoryWrapper;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.store.NIOFSDirectory;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util._TestUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
/*
Verify we can read the pre-5.0 file format, do searches
@ -77,7 +76,7 @@ import org.junit.Ignore;
// we won't even be running the actual code, only the impostor
// @SuppressCodecs("Lucene4x")
// Sep codec cannot yet handle the offsets in our 4.x index!
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41", "Lucene42"})
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41", "Lucene42", "Lucene45"})
public class TestBackwardsCompatibility extends LuceneTestCase {
// Uncomment these cases & run them on an older Lucene version,

View File

@ -29,7 +29,6 @@ import org.apache.lucene.document.TextField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.FailOnNonBulkMergesInfoStream;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.Test;
public class TestConsistentFieldNumbers extends LuceneTestCase {
@ -67,8 +66,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
sis.read(dir);
assertEquals(2, sis.size());
FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0).info);
FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1).info);
FieldInfos fis1 = SegmentReader.readFieldInfos(sis.info(0));
FieldInfos fis2 = SegmentReader.readFieldInfos(sis.info(1));
assertEquals("f1", fis1.fieldInfo(0).name);
assertEquals("f2", fis1.fieldInfo(1).name);
@ -85,7 +84,7 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
sis.read(dir);
assertEquals(1, sis.size());
FieldInfos fis3 = _TestUtil.getFieldInfos(sis.info(0).info);
FieldInfos fis3 = SegmentReader.readFieldInfos(sis.info(0));
assertEquals("f1", fis3.fieldInfo(0).name);
assertEquals("f2", fis3.fieldInfo(1).name);
@ -130,8 +129,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
sis.read(dir1);
assertEquals(2, sis.size());
FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0).info);
FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1).info);
FieldInfos fis1 = SegmentReader.readFieldInfos(sis.info(0));
FieldInfos fis2 = SegmentReader.readFieldInfos(sis.info(1));
assertEquals("f1", fis1.fieldInfo(0).name);
assertEquals("f2", fis1.fieldInfo(1).name);
@ -161,7 +160,7 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
SegmentInfos sis = new SegmentInfos();
sis.read(dir);
assertEquals(1, sis.size());
FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0).info);
FieldInfos fis1 = SegmentReader.readFieldInfos(sis.info(0));
assertEquals("f1", fis1.fieldInfo(0).name);
assertEquals("f2", fis1.fieldInfo(1).name);
}
@ -180,8 +179,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
SegmentInfos sis = new SegmentInfos();
sis.read(dir);
assertEquals(2, sis.size());
FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0).info);
FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1).info);
FieldInfos fis1 = SegmentReader.readFieldInfos(sis.info(0));
FieldInfos fis2 = SegmentReader.readFieldInfos(sis.info(1));
assertEquals("f1", fis1.fieldInfo(0).name);
assertEquals("f2", fis1.fieldInfo(1).name);
assertEquals("f1", fis2.fieldInfo(0).name);
@ -203,9 +202,9 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
SegmentInfos sis = new SegmentInfos();
sis.read(dir);
assertEquals(3, sis.size());
FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0).info);
FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1).info);
FieldInfos fis3 = _TestUtil.getFieldInfos(sis.info(2).info);
FieldInfos fis1 = SegmentReader.readFieldInfos(sis.info(0));
FieldInfos fis2 = SegmentReader.readFieldInfos(sis.info(1));
FieldInfos fis3 = SegmentReader.readFieldInfos(sis.info(2));
assertEquals("f1", fis1.fieldInfo(0).name);
assertEquals("f2", fis1.fieldInfo(1).name);
assertEquals("f1", fis2.fieldInfo(0).name);
@ -237,7 +236,7 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
SegmentInfos sis = new SegmentInfos();
sis.read(dir);
assertEquals(1, sis.size());
FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0).info);
FieldInfos fis1 = SegmentReader.readFieldInfos(sis.info(0));
assertEquals("f1", fis1.fieldInfo(0).name);
assertEquals("f2", fis1.fieldInfo(1).name);
assertEquals("f3", fis1.fieldInfo(2).name);
@ -275,7 +274,7 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
SegmentInfos sis = new SegmentInfos();
sis.read(dir);
for (SegmentInfoPerCommit si : sis) {
FieldInfos fis = _TestUtil.getFieldInfos(si.info);
FieldInfos fis = SegmentReader.readFieldInfos(si);
for (FieldInfo fi : fis) {
Field expected = getField(Integer.parseInt(fi.name));

View File

@ -50,7 +50,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
public void setUp() throws Exception {
super.setUp();
// for now its SimpleText vs Lucene45(random postings format)
// for now its SimpleText vs Lucene46(random postings format)
// as this gives the best overall coverage. when we have more
// codecs we should probably pick 2 from Codec.availableCodecs()

View File

@ -17,19 +17,16 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.util.LuceneTestCase;
//import org.cnlp.utils.properties.ResourceBundleHelper;
@ -60,14 +57,14 @@ public class TestFieldInfos extends LuceneTestCase {
//Use a RAMOutputStream
FieldInfosWriter writer = Codec.getDefault().fieldInfosFormat().getFieldInfosWriter();
writer.write(dir, filename, fieldInfos, IOContext.DEFAULT);
writer.write(dir, filename, "", fieldInfos, IOContext.DEFAULT);
output.close();
return fieldInfos;
}
public FieldInfos readFieldInfos(Directory dir, String filename) throws IOException {
FieldInfosReader reader = Codec.getDefault().fieldInfosFormat().getFieldInfosReader();
return reader.read(dir, filename, IOContext.DEFAULT);
return reader.read(dir, filename, "", IOContext.DEFAULT);
}
public void test() throws IOException {

View File

@ -1,7 +1,9 @@
package org.apache.lucene.index;
import java.io.IOException;
import java.util.HashSet;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import org.apache.lucene.analysis.MockAnalyzer;
@ -11,8 +13,9 @@ import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
import org.apache.lucene.codecs.lucene45.Lucene45RWCodec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
@ -25,9 +28,12 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.junit.Test;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -45,7 +51,7 @@ import org.junit.Test;
* limitations under the License.
*/
@SuppressCodecs({"Lucene40","Lucene41","Lucene42"})
@SuppressCodecs({"Lucene40","Lucene41","Lucene42","Lucene45"})
public class TestNumericDocValuesUpdates extends LuceneTestCase {
private Document doc(int id) {
@ -517,7 +523,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
public void testDifferentDVFormatPerField() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setCodec(new Lucene45Codec() {
conf.setCodec(new Lucene46Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new Lucene45DocValuesFormat();
@ -792,13 +798,17 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
// update document in the second segment
writer.updateNumericDocValue(new Term("id", "doc1"), "ndv", 5L);
try {
writer.close();
fail("should not have succeeded updating a segment with no numeric DocValues field");
} catch (UnsupportedOperationException e) {
// expected
writer.rollback();
DirectoryReader reader = DirectoryReader.open(dir);
for (AtomicReaderContext context : reader.leaves()) {
AtomicReader r = context.reader();
NumericDocValues ndv = r.getNumericDocValues("ndv");
for (int i = 0; i < r.maxDoc(); i++) {
assertEquals(5L, ndv.get(i));
}
}
reader.close();
dir.close();
}
@ -828,15 +838,19 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
writer.addDocument(doc);
writer.commit();
// update documentin the second segment
// update document in the second segment
writer.updateNumericDocValue(new Term("id", "doc1"), "ndv", 5L);
try {
writer.close();
fail("should not have succeeded updating a segment with no numeric DocValues field");
} catch (UnsupportedOperationException e) {
// expected
writer.rollback();
DirectoryReader reader = DirectoryReader.open(dir);
for (AtomicReaderContext context : reader.leaves()) {
AtomicReader r = context.reader();
NumericDocValues ndv = r.getNumericDocValues("ndv");
for (int i = 0; i < r.maxDoc(); i++) {
assertEquals(5L, ndv.get(i));
}
}
reader.close();
dir.close();
}
@ -867,7 +881,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
@Test
public void testUpdateOldSegments() throws Exception {
Codec[] oldCodecs = new Codec[] { new Lucene40RWCodec(), new Lucene41RWCodec(), new Lucene42RWCodec() };
Codec[] oldCodecs = new Codec[] { new Lucene40RWCodec(), new Lucene41RWCodec(), new Lucene42RWCodec(), new Lucene45RWCodec() };
Directory dir = newDirectory();
// create a segment with an old Codec
@ -1038,7 +1052,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setMergePolicy(NoMergePolicy.COMPOUND_FILES); // disable merges to simplify test assertions.
conf.setCodec(new Lucene45Codec() {
conf.setCodec(new Lucene46Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new Lucene45DocValuesFormat();
@ -1053,7 +1067,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
writer.close();
// change format
conf.setCodec(new Lucene45Codec() {
conf.setCodec(new Lucene46Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new AssertingDocValuesFormat();
@ -1080,4 +1094,63 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
dir.close();
}
@Test
public void testAddIndexes() throws Exception {
Directory dir1 = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
IndexWriter writer = new IndexWriter(dir1, conf);
final int numDocs = atLeast(50);
final int numTerms = _TestUtil.nextInt(random(), 1, numDocs / 5);
Set<String> randomTerms = new HashSet<String>();
while (randomTerms.size() < numTerms) {
randomTerms.add(_TestUtil.randomSimpleString(random()));
}
// create first index
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
doc.add(new StringField("id", RandomPicks.randomFrom(random(), randomTerms), Store.NO));
doc.add(new NumericDocValuesField("ndv", 4L));
doc.add(new NumericDocValuesField("control", 8L));
writer.addDocument(doc);
}
if (random().nextBoolean()) {
writer.commit();
}
// update some docs to a random value
long value = random().nextInt();
Term term = new Term("id", RandomPicks.randomFrom(random(), randomTerms));
writer.updateNumericDocValue(term, "ndv", value);
writer.updateNumericDocValue(term, "control", value * 2);
writer.close();
Directory dir2 = newDirectory();
conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
writer = new IndexWriter(dir2, conf);
if (random().nextBoolean()) {
writer.addIndexes(dir1);
} else {
DirectoryReader reader = DirectoryReader.open(dir1);
writer.addIndexes(reader);
reader.close();
}
writer.close();
DirectoryReader reader = DirectoryReader.open(dir2);
for (AtomicReaderContext context : reader.leaves()) {
AtomicReader r = context.reader();
NumericDocValues ndv = r.getNumericDocValues("ndv");
NumericDocValues control = r.getNumericDocValues("control");
for (int i = 0; i < r.maxDoc(); i++) {
assertEquals(ndv.get(i)*2, control.get(i));
}
}
reader.close();
IOUtils.close(dir1, dir2);
}
}

View File

@ -127,7 +127,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
seg = writer.newestSegment();
writer.close();
fieldInfos = _TestUtil.getFieldInfos(seg.info);
fieldInfos = SegmentReader.readFieldInfos(seg);
}
@Override

View File

@ -24,21 +24,22 @@ import org.apache.lucene.codecs.Codec;
// TODO: maybe we should test this with mocks, but its easy
// enough to test the basics via Codec
public class TestNamedSPILoader extends LuceneTestCase {
public void testLookup() {
Codec codec = Codec.forName("Lucene45");
assertEquals("Lucene45", codec.getName());
Codec codec = Codec.forName("Lucene46");
assertEquals("Lucene46", codec.getName());
}
// we want an exception if its not found.
public void testBogusLookup() {
try {
Codec codec = Codec.forName("dskfdskfsdfksdfdsf");
Codec.forName("dskfdskfsdfksdfdsf");
fail();
} catch (IllegalArgumentException expected) {}
}
public void testAvailableServices() {
Set<String> codecs = Codec.availableCodecs();
assertTrue(codecs.contains("Lucene45"));
assertTrue(codecs.contains("Lucene46"));
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.facet.codecs.facet45;
package org.apache.lucene.facet.codecs.facet46;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,13 +21,13 @@ import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.facet.codecs.facet42.Facet42DocValuesFormat;
import org.apache.lucene.facet.params.CategoryListParams;
import org.apache.lucene.facet.params.FacetIndexingParams;
/**
* Same as {@link Lucene45Codec} except it uses {@link Facet42DocValuesFormat}
* Same as {@link Lucene46Codec} except it uses {@link Facet42DocValuesFormat}
* for facet fields (faster-but-more-RAM-consuming doc values).
*
* <p>
@ -43,13 +43,13 @@ import org.apache.lucene.facet.params.FacetIndexingParams;
*
* @lucene.experimental
*/
public class Facet45Codec extends Lucene45Codec {
public class Facet46Codec extends Lucene46Codec {
private final Set<String> facetFields;
private final DocValuesFormat facetsDVFormat = DocValuesFormat.forName("Facet42");
/** Default constructor, uses {@link FacetIndexingParams#DEFAULT}. */
public Facet45Codec() {
public Facet46Codec() {
this(FacetIndexingParams.DEFAULT);
}
@ -58,7 +58,7 @@ public class Facet45Codec extends Lucene45Codec {
* {@link DocValuesFormat} for the fields that are returned by
* {@link FacetIndexingParams#getAllCategoryListParams()}.
*/
public Facet45Codec(FacetIndexingParams fip) {
public Facet46Codec(FacetIndexingParams fip) {
if (fip.getPartitionSize() != Integer.MAX_VALUE) {
throw new IllegalArgumentException("this Codec does not support partitions");
}

View File

@ -3,7 +3,7 @@ package org.apache.lucene.facet;
import java.util.Random;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
import org.apache.lucene.facet.codecs.facet46.Facet46Codec;
import org.apache.lucene.facet.encoding.DGapIntEncoder;
import org.apache.lucene.facet.encoding.DGapVInt8IntEncoder;
import org.apache.lucene.facet.encoding.EightFlagsIntEncoder;
@ -53,7 +53,7 @@ public abstract class FacetTestCase extends LuceneTestCase {
public static void beforeClassFacetTestCase() throws Exception {
if (random().nextDouble() < 0.3) {
savedDefault = Codec.getDefault(); // save to restore later
Codec.setDefault(new Facet45Codec());
Codec.setDefault(new Facet46Codec());
}
}

View File

@ -31,7 +31,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.facet.FacetTestCase;
import org.apache.lucene.facet.FacetTestUtils;
import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
import org.apache.lucene.facet.codecs.facet46.Facet46Codec;
import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.params.CategoryListParams;
import org.apache.lucene.facet.params.FacetIndexingParams;
@ -260,7 +260,7 @@ public class TestDemoFacets extends FacetTestCase {
Directory dir = newDirectory();
Directory taxoDir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(new Facet45Codec());
iwc.setCodec(new Facet46Codec());
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);

View File

@ -141,7 +141,7 @@ public class IndexSplitter {
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(),
info.getUseCompoundFile(),
info.getCodec(), info.getDiagnostics(), info.attributes());
destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getDocValuesGen()));
destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen()));
// now copy files over
Collection<String> files = infoPerCommit.files();
for (final String srcName : files) {

View File

@ -34,7 +34,7 @@ import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -164,7 +164,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
* codec to use. */
protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer) {
IndexWriterConfig iwc = new IndexWriterConfig(matchVersion, indexAnalyzer);
iwc.setCodec(new Lucene45Codec());
iwc.setCodec(new Lucene46Codec());
iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
return iwc;
}

View File

@ -23,10 +23,10 @@ import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
/**
* Acts like {@link Lucene45Codec} but with additional asserts.
* Acts like {@link Lucene46Codec} but with additional asserts.
*/
public final class AssertingCodec extends FilterCodec {
@ -37,7 +37,7 @@ public final class AssertingCodec extends FilterCodec {
private final NormsFormat norms = new AssertingNormsFormat();
public AssertingCodec() {
super("Asserting", new Lucene45Codec());
super("Asserting", new Lucene46Codec());
}
@Override

View File

@ -28,7 +28,7 @@ import org.apache.lucene.codecs.diskdv.DiskNormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
/** Codec that tries to use as little ram as possible because he spent all his money on beer */
// TODO: better name :)
@ -45,9 +45,10 @@ public class CheapBastardCodec extends FilterCodec {
private final NormsFormat norms = new DiskNormsFormat();
public CheapBastardCodec() {
super("CheapBastard", new Lucene45Codec());
super("CheapBastard", new Lucene46Codec());
}
@Override
public PostingsFormat postingsFormat() {
return postings;
}

View File

@ -23,13 +23,13 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
/**
* A codec that uses {@link CompressingStoredFieldsFormat} for its stored
* fields and delegates to {@link Lucene45Codec} for everything else.
* fields and delegates to {@link Lucene46Codec} for everything else.
*/
public abstract class CompressingCodec extends FilterCodec {
@ -73,7 +73,7 @@ public abstract class CompressingCodec extends FilterCodec {
* Creates a compressing codec with a given segment suffix
*/
public CompressingCodec(String name, String segmentSuffix, CompressionMode compressionMode, int chunkSize) {
super(name, new Lucene45Codec());
super(name, new Lucene46Codec());
this.storedFieldsFormat = new CompressingStoredFieldsFormat(name, segmentSuffix, compressionMode, chunkSize);
this.termVectorsFormat = new CompressingTermVectorsFormat(name, segmentSuffix, compressionMode, chunkSize);
}

View File

@ -37,6 +37,7 @@ import org.apache.lucene.util.IOUtils;
* @see Lucene40FieldInfosFormat
* @lucene.experimental
*/
@Deprecated
public class Lucene40FieldInfosWriter extends FieldInfosWriter {
/** Sole constructor. */
@ -44,7 +45,7 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
}
@Override
public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosFormat.FIELD_INFOS_EXTENSION);
IndexOutput output = directory.createOutput(fileName, context);
boolean success = false;

View File

@ -37,14 +37,15 @@ import org.apache.lucene.util.IOUtils;
* @see Lucene42FieldInfosFormat
* @lucene.experimental
*/
final class Lucene42FieldInfosWriter extends FieldInfosWriter {
@Deprecated
public final class Lucene42FieldInfosWriter extends FieldInfosWriter {
/** Sole constructor. */
public Lucene42FieldInfosWriter() {
}
@Override
public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene42FieldInfosFormat.EXTENSION);
IndexOutput output = directory.createOutput(fileName, context);
boolean success = false;

View File

@ -17,12 +17,18 @@ package org.apache.lucene.codecs.lucene42;
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.util.LuceneTestCase;
/**
* Read-write version of {@link Lucene42Codec} for testing.
*/
@SuppressWarnings("deprecation")
public class Lucene42RWCodec extends Lucene42Codec {
private static final DocValuesFormat dv = new Lucene42RWDocValuesFormat();
private static final NormsFormat norms = new Lucene42NormsFormat();
@ -36,4 +42,19 @@ public class Lucene42RWCodec extends Lucene42Codec {
public NormsFormat normsFormat() {
return norms;
}
@Override
public FieldInfosFormat fieldInfosFormat() {
return new Lucene42FieldInfosFormat() {
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
if (!LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
return super.getFieldInfosWriter();
} else {
return new Lucene42FieldInfosWriter();
}
}
};
}
}

View File

@ -0,0 +1,48 @@
package org.apache.lucene.codecs.lucene45;
/*
* 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.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat;
import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosWriter;
import org.apache.lucene.util.LuceneTestCase;
/**
* Read-write version of {@link Lucene45Codec} for testing.
*/
@SuppressWarnings("deprecation")
public class Lucene45RWCodec extends Lucene45Codec {
@Override
public FieldInfosFormat fieldInfosFormat() {
return new Lucene42FieldInfosFormat() {
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
if (!LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
return super.getFieldInfosWriter();
} else {
return new Lucene42FieldInfosWriter();
}
}
};
}
}

View File

@ -0,0 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Support for testing {@link org.apache.lucene.codecs.lucene45.Lucene45Codec}.
</body>
</html>

View File

@ -40,7 +40,7 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -1386,7 +1386,7 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
// TODO: would be better to use / delegate to the current
// Codec returned by getCodec()
iwc.setCodec(new Lucene45Codec() {
iwc.setCodec(new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {

View File

@ -31,21 +31,20 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.compressing.CompressingCodec;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoubleField;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.FieldType.NumericType;
import org.apache.lucene.document.FloatField;
import org.apache.lucene.document.IntField;
import org.apache.lucene.document.LongField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.FieldType.NumericType;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.NumericRangeQuery;
@ -59,7 +58,6 @@ import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.LuceneTestCase.Nightly;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
@ -78,6 +76,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends LuceneTestCase {
*/
protected abstract Codec getCodec();
@Override
public void setUp() throws Exception {
super.setUp();
// set the default codec, so adding test cases to this isn't fragile
@ -85,6 +84,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends LuceneTestCase {
Codec.setDefault(getCodec());
}
@Override
public void tearDown() throws Exception {
Codec.setDefault(savedCodec); // restore
super.tearDown();
@ -502,7 +502,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends LuceneTestCase {
// get another codec, other than the default: so we are merging segments across different codecs
final Codec otherCodec;
if ("SimpleText".equals(Codec.getDefault().getName())) {
otherCodec = new Lucene45Codec();
otherCodec = new Lucene46Codec();
} else {
otherCodec = new SimpleTextCodec();
}

View File

@ -27,19 +27,23 @@ import java.util.Map;
import java.util.Random;
import java.util.Set;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
import org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat;
import org.apache.lucene.codecs.memory.FSTPostingsFormat;
import org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat;
import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
@ -48,12 +52,8 @@ import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat;
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
import org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat;
import org.apache.lucene.codecs.memory.FSTPostingsFormat;
import org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
@ -66,7 +66,7 @@ import org.apache.lucene.util._TestUtil;
* documents in different orders and the test will still be deterministic
* and reproducable.
*/
public class RandomCodec extends Lucene45Codec {
public class RandomCodec extends Lucene46Codec {
/** Shuffled list of postings formats to use for new mappings */
private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();

View File

@ -1386,7 +1386,7 @@ public abstract class LuceneTestCase extends Assert {
/** Returns true if the codec "supports" field updates. */
public static boolean defaultCodecSupportsFieldUpdates() {
String name = Codec.getDefault().getName();
if (name.equals("Lucene40") || name.equals("Lucene41") || name.equals("Lucene42")) {
if (name.equals("Lucene40") || name.equals("Lucene41") || name.equals("Lucene42") || name.equals("Lucene45")) {
return false;
}
return true;

View File

@ -17,14 +17,22 @@ package org.apache.lucene.util;
* limitations under the License.
*/
import static org.apache.lucene.util.LuceneTestCase.INFOSTREAM;
import static org.apache.lucene.util.LuceneTestCase.TEST_CODEC;
import static org.apache.lucene.util.LuceneTestCase.TEST_DOCVALUESFORMAT;
import static org.apache.lucene.util.LuceneTestCase.TEST_POSTINGSFORMAT;
import static org.apache.lucene.util.LuceneTestCase.VERBOSE;
import static org.apache.lucene.util.LuceneTestCase.assumeFalse;
import static org.apache.lucene.util.LuceneTestCase.localeForName;
import static org.apache.lucene.util.LuceneTestCase.random;
import static org.apache.lucene.util.LuceneTestCase.randomLocale;
import static org.apache.lucene.util.LuceneTestCase.randomTimeZone;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
@ -40,20 +48,18 @@ import org.apache.lucene.codecs.compressing.CompressingCodec;
import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.RandomCodec;
import org.apache.lucene.search.RandomSimilarityProvider;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; // javadocs
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.junit.internal.AssumptionViolatedException;
import com.carrotsearch.randomizedtesting.RandomizedContext;
import static org.apache.lucene.util.LuceneTestCase.*;
import com.carrotsearch.randomizedtesting.RandomizedContext;
@ -192,7 +198,7 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
dvFormat = DocValuesFormat.forName(TEST_DOCVALUESFORMAT);
}
codec = new Lucene45Codec() {
codec = new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;

View File

@ -44,7 +44,7 @@ import java.util.zip.ZipFile;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@ -59,18 +59,17 @@ import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
import org.apache.lucene.index.CheckIndex.Status.FieldNormStatus;
import org.apache.lucene.index.CheckIndex.Status.StoredFieldStatus;
import org.apache.lucene.index.CheckIndex.Status.TermIndexStatus;
import org.apache.lucene.index.CheckIndex.Status.TermVectorStatus;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexableField;
@ -78,7 +77,8 @@ import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfoPerCommit;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TieredMergePolicy;
@ -87,9 +87,7 @@ import org.apache.lucene.search.FilteredQuery;
import org.apache.lucene.search.FilteredQuery.FilterStrategy;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.junit.Assert;
import com.carrotsearch.randomizedtesting.RandomizedContext;
@ -703,7 +701,7 @@ public class _TestUtil {
if (LuceneTestCase.VERBOSE) {
System.out.println("forcing postings format to:" + format);
}
return new Lucene45Codec() {
return new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;
@ -721,7 +719,7 @@ public class _TestUtil {
if (LuceneTestCase.VERBOSE) {
System.out.println("forcing docvalues format to:" + format);
}
return new Lucene45Codec() {
return new Lucene46Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return format;
@ -999,27 +997,6 @@ public class _TestUtil {
}
}
public static FieldInfos getFieldInfos(SegmentInfo info) throws IOException {
Directory cfsDir = null;
try {
if (info.getUseCompoundFile()) {
cfsDir = new CompoundFileDirectory(info.dir,
IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
IOContext.READONCE,
false);
} else {
cfsDir = info.dir;
}
return info.getCodec().fieldInfosFormat().getFieldInfosReader().read(cfsDir,
info.name,
IOContext.READONCE);
} finally {
if (info.getUseCompoundFile() && cfsDir != null) {
cfsDir.close();
}
}
}
/**
* Returns a valid (compiling) Pattern instance with random stuff inside. Be careful
* when applying random patterns to longer strings as certain types of patterns

View File

@ -22,3 +22,4 @@ org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec
org.apache.lucene.codecs.lucene40.Lucene40RWCodec
org.apache.lucene.codecs.lucene41.Lucene41RWCodec
org.apache.lucene.codecs.lucene42.Lucene42RWCodec
org.apache.lucene.codecs.lucene45.Lucene45RWCodec

View File

@ -3,7 +3,7 @@ package org.apache.solr.core;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.util.plugin.SolrCoreAware;
@ -51,7 +51,7 @@ public class SchemaCodecFactory extends CodecFactory implements SolrCoreAware {
@Override
public void init(NamedList args) {
super.init(args);
codec = new Lucene45Codec() {
codec = new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
final SchemaField fieldOrNull = core.getLatestSchema().getFieldOrNull(field);