Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java (revision 1525404) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java (working copy) @@ -47,8 +47,8 @@ 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(); @@ -105,6 +105,10 @@ 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)); Map atts = new HashMap(); @@ -122,6 +126,7 @@ 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()) { Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java (revision 1525404) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java (working copy) @@ -53,6 +53,7 @@ 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 "); @@ -59,8 +60,8 @@ 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; @@ -108,6 +109,10 @@ SimpleTextUtil.write(out, DOCVALUES); 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 atts = fi.attributes(); int numAtts = atts == null ? 0 : atts.size(); Index: lucene/core/src/java/org/apache/lucene/codecs/Codec.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/Codec.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/Codec.java (working copy) @@ -119,7 +119,7 @@ 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. Index: lucene/core/src/java/org/apache/lucene/codecs/FieldInfosReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/FieldInfosReader.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/FieldInfosReader.java (working copy) @@ -35,5 +35,5 @@ /** 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; } Index: lucene/core/src/java/org/apache/lucene/codecs/FieldInfosWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/FieldInfosWriter.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/FieldInfosWriter.java (working copy) @@ -35,5 +35,5 @@ /** 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; } Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java (working copy) @@ -49,7 +49,7 @@ } @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); Index: lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java (working copy) @@ -95,7 +95,7 @@ } @Override - public final FieldInfosFormat fieldInfosFormat() { + public FieldInfosFormat fieldInfosFormat() { return fieldInfosFormat; } Index: lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java (working copy) @@ -84,9 +84,9 @@ * * @lucene.experimental */ -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 +99,7 @@ @Override public FieldInfosWriter getFieldInfosWriter() throws IOException { - return writer; + throw new UnsupportedOperationException("this codec can only be used for reading"); } /** Extension of field infos */ Index: lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java (working copy) @@ -40,6 +40,7 @@ * @lucene.experimental * @see Lucene42FieldInfosFormat */ +@Deprecated final class Lucene42FieldInfosReader extends FieldInfosReader { /** Sole constructor. */ @@ -47,7 +48,7 @@ } @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); Index: lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (working copy) @@ -1,108 +0,0 @@ -package org.apache.lucene.codecs.lucene42; - -/* - * 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.2 FieldInfos writer. - * - * @see Lucene42FieldInfosFormat - * @lucene.experimental - */ -final class Lucene42FieldInfosWriter extends FieldInfosWriter { - - /** Sole constructor. */ - public Lucene42FieldInfosWriter() { - } - - @Override - public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException { - final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene42FieldInfosFormat.EXTENSION); - IndexOutput output = directory.createOutput(fileName, context); - boolean success = false; - try { - CodecUtil.writeHeader(output, Lucene42FieldInfosFormat.CODEC_NAME, Lucene42FieldInfosFormat.FORMAT_CURRENT); - output.writeVInt(infos.size()); - for (FieldInfo fi : infos) { - IndexOptions indexOptions = fi.getIndexOptions(); - byte bits = 0x0; - if (fi.hasVectors()) bits |= Lucene42FieldInfosFormat.STORE_TERMVECTOR; - if (fi.omitsNorms()) bits |= Lucene42FieldInfosFormat.OMIT_NORMS; - if (fi.hasPayloads()) bits |= Lucene42FieldInfosFormat.STORE_PAYLOADS; - if (fi.isIndexed()) { - bits |= Lucene42FieldInfosFormat.IS_INDEXED; - assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads(); - if (indexOptions == IndexOptions.DOCS_ONLY) { - bits |= Lucene42FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS; - } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) { - bits |= Lucene42FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS; - } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) { - bits |= Lucene42FieldInfosFormat.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.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(); - } - } -} Index: lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java (working copy) @@ -49,6 +49,7 @@ // 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 +93,7 @@ } @Override - public final FieldInfosFormat fieldInfosFormat() { + public FieldInfosFormat fieldInfosFormat() { return fieldInfosFormat; } Index: lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java (working copy) @@ -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. + *

+ * 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 field. + * + * 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 field. + * + * 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; + } +} Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Added: svn:executable ## -0,0 +1 ## +* \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java (working copy) @@ -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. + *

+ *

Field names are stored in the field info file, with suffix .fnm.

+ *

FieldInfos (.fnm) --> Header,FieldsCount, <FieldName,FieldNumber, + * FieldBits,DocValuesBits,DocValuesGen,Attributes> FieldsCount

+ *

Data types: + *

    + *
  • Header --> {@link CodecUtil#checkHeader CodecHeader}
  • + *
  • FieldsCount --> {@link DataOutput#writeVInt VInt}
  • + *
  • FieldName --> {@link DataOutput#writeString String}
  • + *
  • FieldBits, DocValuesBits --> {@link DataOutput#writeByte Byte}
  • + *
  • FieldNumber --> {@link DataOutput#writeInt VInt}
  • + *
  • Attributes --> {@link DataOutput#writeStringStringMap Map<String,String>}
  • + *
  • DocValuesGen --> {@link DataOutput#writeLong(long) Int64}
  • + *
+ *

+ * Field Descriptions: + *
    + *
  • FieldsCount: the number of fields in this file.
  • + *
  • FieldName: name of the field as a UTF-8 String.
  • + *
  • 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.
  • + *
  • FieldBits: a byte containing field options. + *
      + *
    • The low-order bit is one for indexed fields, and zero for non-indexed + * fields.
    • + *
    • The second lowest-order bit is one for fields that have term vectors + * stored, and zero for fields without term vectors.
    • + *
    • If the third lowest order-bit is set (0x4), offsets are stored into + * the postings list in addition to positions.
    • + *
    • Fourth bit is unused.
    • + *
    • If the fifth lowest-order bit is set (0x10), norms are omitted for the + * indexed field.
    • + *
    • If the sixth lowest-order bit is set (0x20), payloads are stored for the + * indexed field.
    • + *
    • If the seventh lowest-order bit is set (0x40), term frequencies and + * positions omitted for the indexed field.
    • + *
    • If the eighth lowest-order bit is set (0x80), positions are omitted for the + * indexed field.
    • + *
    + *
  • + *
  • DocValuesBits: a byte containing per-document value types. The type + * recorded as two four-bit integers, with the high-order bits representing + * norms options, and the low-order bits representing + * {@code DocValues} options. Each four-bit integer can be decoded as such: + *
      + *
    • 0: no DocValues for this field.
    • + *
    • 1: NumericDocValues. ({@link DocValuesType#NUMERIC})
    • + *
    • 2: BinaryDocValues. ({@code DocValuesType#BINARY})
    • + *
    • 3: SortedDocValues. ({@code DocValuesType#SORTED})
    • + *
    + *
  • + *
  • 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}.
  • + *
  • Attributes: a key-value map of codec-private attributes.
  • + *
+ * + * @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; +} Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Added: svn:executable ## -0,0 +1 ## +* \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java (working copy) @@ -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 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 + ")"); + } + } +} Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Added: svn:executable ## -0,0 +1 ## +* \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java (working copy) @@ -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(); + } + } +} Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Added: svn:executable ## -0,0 +1 ## +* \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/codecs/lucene46/package.html =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene46/package.html (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene46/package.html (working copy) @@ -0,0 +1,398 @@ + + + + + + + +Lucene 4.6 file format. + +

Apache Lucene - Index File Formats

+
+ +
+ +

Introduction

+
+

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 +docs/ that was distributed with +the version you are using.

+

Apache Lucene is written in Java, but several efforts are underway to write +versions of +Lucene in other programming languages. 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.

+

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.

+
+ +

Definitions

+
+

The fundamental concepts in Lucene are index, document, field and term.

+

An index contains a sequence of documents.

+
    +
  • A document is a sequence of fields.
  • +
  • A field is a named sequence of terms.
  • +
  • A term is a sequence of bytes.
  • +
+

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.

+ +

Inverted Indexing

+

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 +inverted index. 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.

+ +

Types of Fields

+

In Lucene, fields may be stored, in which case their text is stored +in the index literally, in a non-inverted manner. Fields that are inverted are +called indexed. A field may be both stored and indexed.

+

The text of a field may be tokenized 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.

+

See the {@link org.apache.lucene.document.Field Field} +java docs for more information on Fields.

+ +

Segments

+

Lucene indexes may be composed of multiple sub-indexes, or segments. +Each segment is a fully independent index, which could be searched separately. +Indexes evolve by:

+
    +
  1. Creating new segments for newly added documents.
  2. +
  3. Merging existing segments.
  4. +
+

Searches may involve multiple segments and/or multiple indexes, each index +potentially composed of a set of segments.

+ +

Document Numbers

+

Internally, Lucene refers to documents by an integer document number. +The first document added to an index is numbered zero, and each subsequent +document added gets a number one greater than the previous.

+

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:

+
    +
  • +

    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 base 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.

    +
  • +
  • +

    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.

    +
  • +
+
+ +

Index Structure Overview

+
+

Each segment index maintains the following:

+
    +
  • +{@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, +
  • +
  • +{@link org.apache.lucene.codecs.lucene46.Lucene46FieldInfosFormat Field names}. + This contains the set of field names used in the index. +
  • +
  • +{@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. +
  • +
  • +{@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. +
  • +
  • +{@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) +
  • +
  • +{@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. +
  • +
  • +{@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. +
  • +
  • +{@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 +
  • +
  • +{@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. +
  • +
  • +{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted documents}. +An optional file indicating which documents are deleted. +
  • +
+

Details on each of these are provided in their linked pages.

+
+ +

File Naming

+
+

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)

+

Typically, all segments in an index are stored in a single directory, +although this is not required.

+

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.

+
+ +

Summary of File Extensions

+
+

The following table summarizes the names and extensions of the files in +Lucene:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameExtensionBrief Description
{@link org.apache.lucene.index.SegmentInfos Segments File}segments.gen, segments_NStores information about a commit point
Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same +file.
{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment Info}.siStores metadata about a segment
{@link org.apache.lucene.store.CompoundFileDirectory Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for +systems that frequently run out of file handles.
{@link org.apache.lucene.codecs.lucene46.Lucene46FieldInfosFormat Fields}.fnmStores information about the fields
{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Index}.fdxContains pointers to field data
{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Data}.fdtThe stored fields for documents
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index}.tipThe index into the Term Dictionary
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Positions}.posStores position information about where a term occurs in the index
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
{@link org.apache.lucene.codecs.lucene42.Lucene42NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
{@link org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Documents}.tvdContains information about each document that has term vectors
{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Fields}.tvfThe field level info about term vectors
{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted Documents}.delInfo about what files are deleted
+
+ +

Lock File

+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.

+ +

History

+

Compatibility notes are provided in this document, describing how file +formats have changed from prior versions:

+
    +
  • 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.
  • +
  • In version 2.3, the file format was changed to allow segments to share a +single set of doc store (vectors & 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).
  • +
  • In version 2.4, Strings are now written as true UTF-8 byte sequence, not +Java's modified UTF-8. See +LUCENE-510 for details.
  • +
  • In version 2.9, an optional opaque Map<String,String> CommitUserData +may be passed to IndexWriter's commit methods (and later retrieved), which is +recorded in the segments_N file. See +LUCENE-1382 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 +LUCENE-1654 for details.
  • +
  • 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 LUCENE-1960 +for details.
  • +
  • In version 3.1, segments records the code version that created them. See +LUCENE-2720 for details. +Additionally segments track explicitly whether or not they have term vectors. +See LUCENE-2811 +for details.
  • +
  • In version 3.2, numeric fields are written as natively to stored fields +file, previously they were stored in text format only.
  • +
  • In version 3.4, fields can omit position data while still indexing term +frequencies.
  • +
  • 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.
  • +
  • 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.
  • +
  • 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.
  • +
  • In version 4.5, DocValues were extended to explicitly represent missing values.
  • +
  • In version 4.6, FieldInfos were extended to support per-field DocValues generation, to +allow updating NumericDocValues fields.
  • +
+ +

Limitations

+
+

Lucene uses a Java int to refer to +document numbers, and the index file format uses an Int32 +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 UInt64 values, or +better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.

+
+ + Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene46/package.html ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Added: svn:executable ## -0,0 +1 ## +* \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (working copy) @@ -122,12 +122,15 @@ } 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)"; - format = DocValuesFormat.forName(formatName); - } else { + // this means the field never existed in that segment, yet is applied updates + if (formatName != null) { + format = DocValuesFormat.forName(formatName); + } + } + if (format == null) { format = getDocValuesFormatForField(field.name); } if (format == null) { @@ -136,19 +139,25 @@ 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)"; - suffix = Integer.valueOf(suffixAtt); - } else { + // 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); + } + } + + if (suffix == null) { // bump the suffix suffix = suffixes.get(formatName); if (suffix == null) { @@ -172,7 +181,7 @@ } 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 ... Index: lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (working copy) @@ -85,7 +85,7 @@ // 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 Index: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (working copy) @@ -47,6 +47,8 @@ private Map 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 @@ * * @lucene.experimental */ - public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector, - boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, Map attributes) { + public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector, boolean omitNorms, + boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, + Map attributes) { this.name = name; this.indexed = indexed; this.number = number; @@ -223,7 +226,20 @@ 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. */ public DocValuesType getNormType() { Index: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (working copy) @@ -783,27 +783,6 @@ } } - 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 +791,7 @@ 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 +1526,6 @@ * that already exist in the index, not add new fields through this method. * *

- * NOTE: 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.). - * - *

* NOTE: if this method hits an OutOfMemoryError you should immediately * close the writer. See above for details. *

@@ -1569,7 +1541,6 @@ * @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 +2402,7 @@ 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 +2603,7 @@ // 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 attributes; // copy the attributes map, we might modify it below. @@ -2648,7 +2619,7 @@ 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 segFiles = new HashSet(); Index: lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (working copy) @@ -29,7 +29,7 @@ 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; @@ -370,11 +370,11 @@ // 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 +383,17 @@ 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 +404,9 @@ final Map 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() { @@ -451,6 +448,8 @@ } }); } + + codec.fieldInfosFormat().getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, IOContext.DEFAULT); fieldsConsumerSuccess = true; } finally { if (fieldsConsumerSuccess) { @@ -478,7 +477,7 @@ // 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 +501,7 @@ } 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(); Index: lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (working copy) @@ -50,7 +50,7 @@ // SegmentReaders: private final AtomicInteger ref = new AtomicInteger(1); - final FieldInfos fieldInfos; + private final FieldInfos fieldInfos; final FieldsProducer fields; final DocValuesProducer normsProducer; @@ -103,10 +103,10 @@ cfsDir = dir; } - fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE); + fieldInfos = owner.fieldInfos; + final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context); final PostingsFormat format = codec.postingsFormat(); - final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context); // Ask codec for its Fields fields = format.fieldsProducer(segmentReadState); assert fields != null; Index: lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (working copy) @@ -19,9 +19,7 @@ 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 @@ // 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 fieldDocValuesGens = new HashMap(); + // 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 updatesFiles = new HashSet(); @@ -71,10 +65,10 @@ * 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 @@ 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 @@ 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 @@ /** 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 next available generation number of the FieldInfos files. */ + public long getNextFieldInfosGen() { + return nextWriteFieldInfosGen; } /** - * 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 getFieldDocValuesGens() { - return fieldDocValuesGens; - } - - /** * 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,8 +232,8 @@ if (delGen != -1) { s += ":delGen=" + delGen; } - if (docValuesGen != -1) { - s += ":docValuesGen=" + docValuesGen; + if (fieldInfosGen != -1) { + s += ":fieldInfosGen=" + fieldInfosGen; } return s; } @@ -269,17 +240,16 @@ @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; } } Index: lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (working copy) @@ -28,11 +28,11 @@ 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 @@ *
    *
  • segments.gen: GenHeader, Generation, Generation *
  • segments_N: Header, Version, NameCounter, SegCount, - * <SegName, SegCodec, DelGen, DeletionCount>SegCount, + * <SegName, SegCodec, DelGen, DeletionCount, FieldInfosGen, UpdatesFiles>SegCount, * CommitUserData, Checksum *
*

@@ -79,9 +79,10 @@ *
    *
  • Header --> {@link CodecUtil#writeHeader CodecHeader}
  • *
  • GenHeader, NameCounter, SegCount, DeletionCount --> {@link DataOutput#writeInt Int32}
  • - *
  • Generation, Version, DelGen, Checksum --> {@link DataOutput#writeLong Int64}
  • + *
  • Generation, Version, DelGen, Checksum, FieldInfosGen --> {@link DataOutput#writeLong Int64}
  • *
  • SegName, SegCodec --> {@link DataOutput#writeString String}
  • *
  • CommitUserData --> {@link DataOutput#writeStringStringMap Map<String,String>}
  • + *
  • UpdatesFiles --> {@link DataOutput#writeStringSet(Set) Set<String>}
  • *
*

* Field Descriptions: @@ -104,6 +105,10 @@ *
  • CommitUserData stores an optional user-supplied opaque * Map<String,String> that was passed to * {@link IndexWriter#setCommitData(java.util.Map)}.
  • + *
  • 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}.
  • + *
  • UpdatesFiles stores the list of files that were updated in that segment.
  • * *

    * @@ -338,16 +343,12 @@ if (delCount < 0 || delCount > info.getDocCount()) { throw new CorruptIndexException("invalid deletion count: " + delCount + " (resource: " + input + ")"); } - long docValuesGen = -1; + long fieldInfosGen = -1; if (format >= VERSION_45) { - docValuesGen = input.readLong(); + fieldInfosGen = input.readLong(); } - SegmentInfoPerCommit siPerCommit = new SegmentInfoPerCommit(info, delCount, delGen, docValuesGen); + SegmentInfoPerCommit siPerCommit = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen); if (format >= VERSION_45) { - int numUpdates = input.readInt(); - for (int i = 0; i < numUpdates; i++) { - siPerCommit.setDocValuesGen(input.readInt(), input.readLong()); - } siPerCommit.addUpdatesFiles(input.readStringSet()); } add(siPerCommit); @@ -418,13 +419,7 @@ segnOutput.writeString(si.getCodec().getName()); segnOutput.writeLong(siPerCommit.getDelGen()); segnOutput.writeInt(siPerCommit.getDelCount()); - segnOutput.writeLong(siPerCommit.getDocValuesGen()); - Map docValuesUpdatesGen = siPerCommit.getFieldDocValuesGens(); - segnOutput.writeInt(docValuesUpdatesGen.size()); - for (Entry e : docValuesUpdatesGen.entrySet()) { - segnOutput.writeInt(e.getKey()); - segnOutput.writeLong(e.getValue()); - } + segnOutput.writeLong(siPerCommit.getFieldInfosGen()); segnOutput.writeStringSet(siPerCommit.getUpdatesFiles()); assert si.dir == directory; Index: lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -142,7 +142,7 @@ // 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; } Index: lucene/core/src/java/org/apache/lucene/index/SegmentReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (working copy) @@ -31,6 +31,7 @@ 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 @@ final Map dvProducers = new HashMap(); final Map> genDVProducers = new HashMap>(); + final FieldInfos fieldInfos; + /** * Constructs a new SegmentReader with a new core. * @throws CorruptIndexException if the index is corrupt @@ -81,6 +84,7 @@ // TODO: why is this public? public SegmentReader(SegmentInfoPerCommit si, IOContext context) throws IOException { this.si = si; + fieldInfos = readFieldInfos(si); core = new SegmentCoreReaders(this, si.info.dir, si, context); boolean success = false; @@ -95,7 +99,7 @@ } 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 +161,14 @@ // increment refCount of DocValuesProducers that are used by this reader boolean success = false; try { - if (core.fieldInfos.hasDocValues()) { - final Codec codec = si.info.getCodec(); + 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 +206,45 @@ } } + /** + * Reads the most recent {@link FieldInfos} of the given segment info. + * + * @lucene.internal + */ + public 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 mapping. Fields without DV updates have gen=-1 private Map> getGenInfos(SegmentInfoPerCommit si) { final Map> genInfos = new HashMap>(); - 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 infos = genInfos.get(gen); if (infos == null) { infos = new ArrayList(); @@ -267,7 +308,7 @@ @Override public FieldInfos getFieldInfos() { ensureOpen(); - return core.fieldInfos; + return fieldInfos; } /** Expert: retrieve thread-private {@link @@ -372,7 +413,7 @@ // 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 +455,7 @@ @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; Index: lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (working copy) @@ -71,24 +71,20 @@ * 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 @@ 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 @@ this.segmentSuffix = segmentSuffix; segDeletes = state.segDeletes; delCountOnFlush = state.delCountOnFlush; - isFieldUpdate = state.isFieldUpdate; } } Index: lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (revision 1525404) +++ lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (working copy) @@ -163,7 +163,7 @@ 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: Index: lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec =================================================================== --- lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (revision 1525404) +++ lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (working copy) @@ -16,4 +16,5 @@ org.apache.lucene.codecs.lucene40.Lucene40Codec org.apache.lucene.codecs.lucene41.Lucene41Codec org.apache.lucene.codecs.lucene42.Lucene42Codec -org.apache.lucene.codecs.lucene45.Lucene45Codec \ No newline at end of file +org.apache.lucene.codecs.lucene45.Lucene45Codec +org.apache.lucene.codecs.lucene46.Lucene46Codec \ No newline at end of file Index: lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java =================================================================== --- lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java (working copy) @@ -19,7 +19,7 @@ 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 @@ 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"); Index: lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java =================================================================== --- lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java (working copy) @@ -25,7 +25,7 @@ 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 @@ 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)) { Index: lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java =================================================================== --- lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java (working copy) @@ -21,8 +21,8 @@ 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.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 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 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 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 void testSameCodecDifferentParams() throws Exception { - Codec codec = new Lucene45Codec() { + Codec codec = new Lucene46Codec() { @Override public PostingsFormat getPostingsFormatForField(String field) { if ("id".equals(field)) { Index: lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (working copy) @@ -28,7 +28,7 @@ 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.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 @@ 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 @@ private static final class UnRegisteredCodec extends FilterCodec { public UnRegisteredCodec() { - super("NotRegistered", new Lucene45Codec()); + super("NotRegistered", new Lucene46Codec()); } } Index: lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java (working copy) @@ -21,12 +21,9 @@ 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 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(); Index: lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (working copy) @@ -37,8 +37,8 @@ import org.apache.lucene.document.FieldType; import org.apache.lucene.document.FloatDocValuesField; import org.apache.lucene.document.IntField; +import org.apache.lucene.document.LongField; import org.apache.lucene.document.NumericDocValuesField; -import org.apache.lucene.document.LongField; 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.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; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; -import org.apache.lucene.util.LuceneTestCase; 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 @@ // 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, Index: lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java (working copy) @@ -67,8 +67,8 @@ 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 = _TestUtil.getFieldInfos(sis.info(0)); + FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); @@ -85,7 +85,7 @@ sis.read(dir); assertEquals(1, sis.size()); - FieldInfos fis3 = _TestUtil.getFieldInfos(sis.info(0).info); + FieldInfos fis3 = _TestUtil.getFieldInfos(sis.info(0)); assertEquals("f1", fis3.fieldInfo(0).name); assertEquals("f2", fis3.fieldInfo(1).name); @@ -130,8 +130,8 @@ 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 = _TestUtil.getFieldInfos(sis.info(0)); + FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); @@ -161,7 +161,7 @@ SegmentInfos sis = new SegmentInfos(); sis.read(dir); assertEquals(1, sis.size()); - FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0).info); + FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); } @@ -180,8 +180,8 @@ 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 = _TestUtil.getFieldInfos(sis.info(0)); + FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); assertEquals("f1", fis2.fieldInfo(0).name); @@ -203,9 +203,9 @@ 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 = _TestUtil.getFieldInfos(sis.info(0)); + FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1)); + FieldInfos fis3 = _TestUtil.getFieldInfos(sis.info(2)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); assertEquals("f1", fis2.fieldInfo(0).name); @@ -237,7 +237,7 @@ SegmentInfos sis = new SegmentInfos(); sis.read(dir); assertEquals(1, sis.size()); - FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0).info); + FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); assertEquals("f3", fis1.fieldInfo(2).name); @@ -275,7 +275,7 @@ SegmentInfos sis = new SegmentInfos(); sis.read(dir); for (SegmentInfoPerCommit si : sis) { - FieldInfos fis = _TestUtil.getFieldInfos(si.info); + FieldInfos fis = _TestUtil.getFieldInfos(si); for (FieldInfo fi : fis) { Field expected = getField(Integer.parseInt(fi.name)); Index: lucene/core/src/test/org/apache/lucene/index/TestFieldInfos.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestFieldInfos.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/index/TestFieldInfos.java (working copy) @@ -17,20 +17,17 @@ * 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 org.apache.lucene.util.LuceneTestCase; -import java.io.IOException; -import java.util.Arrays; - //import org.cnlp.utils.properties.ResourceBundleHelper; public class TestFieldInfos extends LuceneTestCase { @@ -60,7 +57,7 @@ //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; } @@ -67,7 +64,7 @@ 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 { Index: lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (working copy) @@ -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,8 @@ 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.lucene46.Lucene46Codec; import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field.Store; @@ -25,9 +27,12 @@ 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 +50,7 @@ * limitations under the License. */ -@SuppressCodecs({"Lucene40","Lucene41","Lucene42"}) +@SuppressCodecs({"Lucene40","Lucene41","Lucene42","Lucene45"}) public class TestNumericDocValuesUpdates extends LuceneTestCase { private Document doc(int id) { @@ -154,7 +159,7 @@ writer.commit(); reader1 = DirectoryReader.open(dir); } - + // update doc writer.updateNumericDocValue(new Term("id", "doc-0"), "val", 10L); // update doc-0's value to 10 if (!isNRT) { @@ -165,7 +170,7 @@ final DirectoryReader reader2 = DirectoryReader.openIfChanged(reader1); assertNotNull(reader2); assertTrue(reader1 != reader2); - + assertEquals(1, reader1.leaves().get(0).reader().getNumericDocValues("val").get(0)); assertEquals(10, reader2.leaves().get(0).reader().getNumericDocValues("val").get(0)); @@ -517,7 +522,7 @@ 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,14 +797,18 @@ // 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(); + writer.close(); + + 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 +837,19 @@ 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(); + writer.close(); + + 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(); } @@ -1037,7 +1050,7 @@ public void testChangeCodec() 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(); @@ -1052,7 +1065,7 @@ writer.close(); // change format - conf.setCodec(new Lucene45Codec() { + conf.setCodec(new Lucene46Codec() { @Override public DocValuesFormat getDocValuesFormatForField(String field) { return new AssertingDocValuesFormat(); @@ -1079,4 +1092,63 @@ 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 randomTerms = new HashSet(); + 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); + } + } Index: lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (revision 1525404) +++ lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (working copy) @@ -127,7 +127,7 @@ seg = writer.newestSegment(); writer.close(); - fieldInfos = _TestUtil.getFieldInfos(seg.info); + fieldInfos = _TestUtil.getFieldInfos(seg); } @Override Index: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java =================================================================== --- lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java (revision 1525404) +++ lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java (working copy) @@ -1,79 +0,0 @@ -package org.apache.lucene.facet.codecs.facet45; - -/* - * 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.util.HashSet; -import java.util.Set; - -import org.apache.lucene.codecs.DocValuesFormat; -import org.apache.lucene.codecs.lucene45.Lucene45Codec; -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} - * for facet fields (faster-but-more-RAM-consuming doc values). - * - *

    - * NOTE: this codec does not support facet partitions (see - * {@link FacetIndexingParams#getPartitionSize()}). - * - *

    - * NOTE: this format cannot handle more than 2 GB - * of facet data in a single segment. If your usage may hit - * this limit, you can either use Lucene's default - * DocValuesFormat, limit the maximum segment size in your - * MergePolicy, or send us a patch fixing the limitation. - * - * @lucene.experimental - */ -public class Facet45Codec extends Lucene45Codec { - - private final Set facetFields; - private final DocValuesFormat facetsDVFormat = DocValuesFormat.forName("Facet42"); - - /** Default constructor, uses {@link FacetIndexingParams#DEFAULT}. */ - public Facet45Codec() { - this(FacetIndexingParams.DEFAULT); - } - - /** - * Initializes with the given {@link FacetIndexingParams}. Returns the proper - * {@link DocValuesFormat} for the fields that are returned by - * {@link FacetIndexingParams#getAllCategoryListParams()}. - */ - public Facet45Codec(FacetIndexingParams fip) { - if (fip.getPartitionSize() != Integer.MAX_VALUE) { - throw new IllegalArgumentException("this Codec does not support partitions"); - } - this.facetFields = new HashSet(); - for (CategoryListParams clp : fip.getAllCategoryListParams()) { - facetFields.add(clp.field); - } - } - - @Override - public DocValuesFormat getDocValuesFormatForField(String field) { - if (facetFields.contains(field)) { - return facetsDVFormat; - } else { - return super.getDocValuesFormatForField(field); - } - } -} Index: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html =================================================================== --- lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html (revision 1525404) +++ lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html (working copy) @@ -1,22 +0,0 @@ - - - - -Codec + DocValuesFormat that are optimized for facets. - - Index: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/Facet46Codec.java =================================================================== --- lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/Facet46Codec.java (revision 0) +++ lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/Facet46Codec.java (working copy) @@ -0,0 +1,79 @@ +package org.apache.lucene.facet.codecs.facet46; + +/* + * 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.util.HashSet; +import java.util.Set; + +import org.apache.lucene.codecs.DocValuesFormat; +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 Lucene46Codec} except it uses {@link Facet42DocValuesFormat} + * for facet fields (faster-but-more-RAM-consuming doc values). + * + *

    + * NOTE: this codec does not support facet partitions (see + * {@link FacetIndexingParams#getPartitionSize()}). + * + *

    + * NOTE: this format cannot handle more than 2 GB + * of facet data in a single segment. If your usage may hit + * this limit, you can either use Lucene's default + * DocValuesFormat, limit the maximum segment size in your + * MergePolicy, or send us a patch fixing the limitation. + * + * @lucene.experimental + */ +public class Facet46Codec extends Lucene46Codec { + + private final Set facetFields; + private final DocValuesFormat facetsDVFormat = DocValuesFormat.forName("Facet42"); + + /** Default constructor, uses {@link FacetIndexingParams#DEFAULT}. */ + public Facet46Codec() { + this(FacetIndexingParams.DEFAULT); + } + + /** + * Initializes with the given {@link FacetIndexingParams}. Returns the proper + * {@link DocValuesFormat} for the fields that are returned by + * {@link FacetIndexingParams#getAllCategoryListParams()}. + */ + public Facet46Codec(FacetIndexingParams fip) { + if (fip.getPartitionSize() != Integer.MAX_VALUE) { + throw new IllegalArgumentException("this Codec does not support partitions"); + } + this.facetFields = new HashSet(); + for (CategoryListParams clp : fip.getAllCategoryListParams()) { + facetFields.add(clp.field); + } + } + + @Override + public DocValuesFormat getDocValuesFormatForField(String field) { + if (facetFields.contains(field)) { + return facetsDVFormat; + } else { + return super.getDocValuesFormatForField(field); + } + } +} Property changes on: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/Facet46Codec.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Added: svn:executable ## -0,0 +1 ## +* \ No newline at end of property Index: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/package.html =================================================================== --- lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/package.html (revision 0) +++ lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/package.html (working copy) @@ -0,0 +1,22 @@ + + + + +Codec + DocValuesFormat that are optimized for facets. + + Property changes on: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/package.html ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Added: svn:executable ## -0,0 +1 ## +* \ No newline at end of property Index: lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java =================================================================== --- lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java (revision 1525404) +++ lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java (working copy) @@ -3,7 +3,7 @@ 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 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()); } } Index: lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java =================================================================== --- lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java (revision 1525404) +++ lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java (working copy) @@ -31,7 +31,7 @@ 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 @@ 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); Index: lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java =================================================================== --- lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (revision 1525404) +++ lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (working copy) @@ -141,7 +141,7 @@ 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 files = infoPerCommit.files(); for (final String srcName : files) { Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (revision 1525404) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (working copy) @@ -34,7 +34,7 @@ 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 @@ * 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; } Index: lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java (working copy) @@ -23,10 +23,10 @@ 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 @@ private final NormsFormat norms = new AssertingNormsFormat(); public AssertingCodec() { - super("Asserting", new Lucene45Codec()); + super("Asserting", new Lucene46Codec()); } @Override Index: lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java (working copy) @@ -28,7 +28,7 @@ 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 @@ private final NormsFormat norms = new DiskNormsFormat(); public CheapBastardCodec() { - super("CheapBastard", new Lucene45Codec()); + super("CheapBastard", new Lucene46Codec()); } + @Override public PostingsFormat postingsFormat() { return postings; } Index: lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java (working copy) @@ -23,13 +23,13 @@ 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 @@ * 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); } Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java (working copy) @@ -37,6 +37,7 @@ * @see Lucene40FieldInfosFormat * @lucene.experimental */ +@Deprecated public class Lucene40FieldInfosWriter extends FieldInfosWriter { /** Sole constructor. */ @@ -44,7 +45,7 @@ } @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; Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (revision 0) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (working copy) @@ -0,0 +1,109 @@ +package org.apache.lucene.codecs.lucene42; + +/* + * 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.2 FieldInfos writer. + * + * @see Lucene42FieldInfosFormat + * @lucene.experimental + */ +@Deprecated +public final class Lucene42FieldInfosWriter extends FieldInfosWriter { + + /** Sole constructor. */ + public Lucene42FieldInfosWriter() { + } + + @Override + 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; + try { + CodecUtil.writeHeader(output, Lucene42FieldInfosFormat.CODEC_NAME, Lucene42FieldInfosFormat.FORMAT_CURRENT); + output.writeVInt(infos.size()); + for (FieldInfo fi : infos) { + IndexOptions indexOptions = fi.getIndexOptions(); + byte bits = 0x0; + if (fi.hasVectors()) bits |= Lucene42FieldInfosFormat.STORE_TERMVECTOR; + if (fi.omitsNorms()) bits |= Lucene42FieldInfosFormat.OMIT_NORMS; + if (fi.hasPayloads()) bits |= Lucene42FieldInfosFormat.STORE_PAYLOADS; + if (fi.isIndexed()) { + bits |= Lucene42FieldInfosFormat.IS_INDEXED; + assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads(); + if (indexOptions == IndexOptions.DOCS_ONLY) { + bits |= Lucene42FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS; + } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) { + bits |= Lucene42FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS; + } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) { + bits |= Lucene42FieldInfosFormat.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.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(); + } + } +} Property changes on: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java (working copy) @@ -17,12 +17,18 @@ * 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. */ +@Deprecated public class Lucene42RWCodec extends Lucene42Codec { private static final DocValuesFormat dv = new Lucene42RWDocValuesFormat(); private static final NormsFormat norms = new Lucene42NormsFormat(); @@ -36,4 +42,19 @@ 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(); + } + } + }; + } + } Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java (revision 0) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java (working copy) @@ -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. + */ +@Deprecated +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(); + } + } + }; + } + +} Property changes on: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/package.html =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/package.html (revision 0) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/package.html (working copy) @@ -0,0 +1,25 @@ + + + + + + + +Support for testing {@link org.apache.lucene.codecs.lucene45.Lucene45Codec}. + + \ No newline at end of file Property changes on: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/package.html ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (working copy) @@ -40,7 +40,7 @@ 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 @@ // 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) { Index: lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (working copy) @@ -31,13 +31,14 @@ 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; @@ -44,8 +45,6 @@ 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.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 @@ */ 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 @@ Codec.setDefault(getCodec()); } + @Override public void tearDown() throws Exception { Codec.setDefault(savedCodec); // restore super.tearDown(); @@ -502,7 +502,7 @@ // 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(); } Index: lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (working copy) @@ -27,19 +27,23 @@ import java.util.Random; import java.util.Set; +import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.DocValuesFormat; 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.mocksep.MockSepPostingsFormat; import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat; import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat; +import org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat; 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.util.LuceneTestCase; import org.apache.lucene.util._TestUtil; @@ -66,7 +66,7 @@ * 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 formats = new ArrayList(); Index: lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (working copy) @@ -1386,7 +1386,7 @@ /** 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; Index: lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (working copy) @@ -17,14 +17,22 @@ * 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,23 +48,21 @@ 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.lucene42.Lucene42RWCodec; +import org.apache.lucene.codecs.lucene46.Lucene46Codec; 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.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.*; - /** * Setup and restore suite-level environment (fine grained junk that * doesn't fit anywhere else). @@ -192,7 +198,7 @@ dvFormat = DocValuesFormat.forName(TEST_DOCVALUESFORMAT); } - codec = new Lucene45Codec() { + codec = new Lucene46Codec() { @Override public PostingsFormat getPostingsFormatForField(String field) { return format; Index: lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (revision 1525404) +++ lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (working copy) @@ -44,7 +44,7 @@ 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.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.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.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 @@ 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 @@ 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,25 +997,8 @@ } } - 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(); - } - } + public static FieldInfos getFieldInfos(SegmentInfoPerCommit info) throws IOException { + return SegmentReader.readFieldInfos(info); } /** Index: lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec =================================================================== --- lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (revision 1525404) +++ lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (working copy) @@ -22,3 +22,4 @@ org.apache.lucene.codecs.lucene40.Lucene40RWCodec org.apache.lucene.codecs.lucene41.Lucene41RWCodec org.apache.lucene.codecs.lucene42.Lucene42RWCodec +org.apache.lucene.codecs.lucene45.Lucene45RWCodec Index: solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java =================================================================== --- solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (revision 1525404) +++ solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (working copy) @@ -3,7 +3,7 @@ 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 @@ @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);