LUCENE-5969: Add Lucene50Codec (infos, dv, norms)

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1627187 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2014-09-24 00:59:50 +00:00
parent 85bf318173
commit 1f7d89668e
19 changed files with 1256 additions and 108 deletions

View File

@ -30,7 +30,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
import org.apache.lucene.codecs.lucene50.Lucene50Codec;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexDeletionPolicy;
@ -139,7 +139,7 @@ public class CreateIndexTask extends PerfTask {
if (defaultCodec == null && postingsFormat != null) {
try {
final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
iwConf.setCodec(new Lucene410Codec(){
iwConf.setCodec(new Lucene50Codec(){
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postingsFormatChosen;

View File

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

View File

@ -30,6 +30,7 @@ import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* Utility class for reading and writing versioned headers.
@ -77,6 +78,7 @@ public final class CodecUtil {
* less than 128 characters in length.
* @param version Version number
* @throws IOException If there is an I/O error writing to the underlying medium.
* @throws IllegalArgumentException If the codec name is not simple ASCII, or is more than 127 characters in length
*/
public static void writeHeader(DataOutput out, String codec, int version)
throws IOException {
@ -211,12 +213,62 @@ public final class CodecUtil {
throw new CorruptIndexException("checksum failed (hardware problem?) : expected=" + Long.toHexString(expectedChecksum) +
" actual=" + Long.toHexString(actualChecksum), in);
}
if (in.getFilePointer() != in.length()) {
throw new CorruptIndexException("did not read all bytes from file: read " + in.getFilePointer() + " vs size " + in.length(), in);
}
return actualChecksum;
}
/**
* Validates the codec footer previously written by {@link #writeFooter}, optionally
* passing an unexpected exception that has already occurred.
* <p>
* When a {@code priorException} is provided, this method will add a suppressed exception
* indicating whether the checksum for the stream passes, fails, or cannot be computed, and
* rethrow it. Otherwise it behaves the same as {@link #checkFooter(ChecksumIndexInput)}.
* <p>
* Example usage:
* <pre class="prettyprint">
* try (ChecksumIndexInput input = ...) {
* Throwable priorE = null;
* try {
* // ... read a bunch of stuff ...
* } catch (Throwable exception) {
* priorE = exception;
* } finally {
* CodecUtil.checkFooter(input, priorE);
* }
* }
* </pre>
*/
public static void checkFooter(ChecksumIndexInput in, Throwable priorException) throws IOException {
if (priorException == null) {
checkFooter(in);
} else {
try {
long remaining = in.length() - in.getFilePointer();
if (remaining < footerLength()) {
// corruption caused us to read into the checksum footer already: we can't proceed
priorException.addSuppressed(new CorruptIndexException("checksum status indeterminate: remaining=" + remaining +
", please run checkindex for more details", in));
} else {
// otherwise, skip any unread bytes.
in.skipBytes(remaining - footerLength());
// now check the footer
try {
long checksum = checkFooter(in);
priorException.addSuppressed(new CorruptIndexException("checksum passed (" + Long.toHexString(checksum) +
"). possibly transient resource issue, or a Lucene or JVM bug", in));
} catch (CorruptIndexException t) {
priorException.addSuppressed(t);
}
}
} catch (Throwable t) {
// catch-all for things that shouldn't go wrong (e.g. OOM during readInt) but could...
priorException.addSuppressed(new CorruptIndexException("checksum status indeterminate: unexpected exception", in, t));
}
IOUtils.reThrow(priorException);
}
}
/**
* Returns (but does not validate) the checksum previously written by {@link #checkFooter}.
* @return actual checksum value
@ -229,6 +281,14 @@ public final class CodecUtil {
}
private static void validateFooter(IndexInput in) throws IOException {
long remaining = in.length() - in.getFilePointer();
long expected = footerLength();
if (remaining < expected) {
throw new CorruptIndexException("misplaced codec footer (file truncated?): remaining=" + remaining + ", expected=" + expected, in);
} else if (remaining > expected) {
throw new CorruptIndexException("misplaced codec footer (file extended?): remaining=" + remaining + ", expected=" + expected, in);
}
final int magic = in.readInt();
if (magic != FOOTER_MAGIC) {
throw new CorruptIndexException("codec footer mismatch: actual footer=" + magic + " vs expected footer=" + FOOTER_MAGIC, in);

View File

@ -76,17 +76,16 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
/** reader for {@link Lucene410DocValuesFormat} */
class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable {
private final Map<String,NumericEntry> numerics;
private final Map<String,BinaryEntry> binaries;
private final Map<String,SortedSetEntry> sortedSets;
private final Map<String,SortedSetEntry> sortedNumerics;
private final Map<String,NumericEntry> ords;
private final Map<String,NumericEntry> ordIndexes;
private final Map<String,NumericEntry> numerics = new HashMap<>();
private final Map<String,BinaryEntry> binaries = new HashMap<>();
private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
private final Map<String,NumericEntry> ords = new HashMap<>();
private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
private final int numFields;
private final AtomicLong ramBytesUsed;
private final IndexInput data;
private final int maxDoc;
private final int version;
// memory-resident structures
private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
@ -98,22 +97,25 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
this.maxDoc = state.segmentInfo.getDocCount();
int version = -1;
int numFields = -1;
// read in the entries from the metadata file.
try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
version = CodecUtil.checkHeader(in, metaCodec,
Lucene410DocValuesFormat.VERSION_START,
Lucene410DocValuesFormat.VERSION_CURRENT);
numerics = new HashMap<>();
ords = new HashMap<>();
ordIndexes = new HashMap<>();
binaries = new HashMap<>();
sortedSets = new HashMap<>();
sortedNumerics = new HashMap<>();
numFields = readFields(in, state.fieldInfos);
CodecUtil.checkFooter(in);
Throwable priorE = null;
try {
version = CodecUtil.checkHeader(in, metaCodec,
Lucene410DocValuesFormat.VERSION_START,
Lucene410DocValuesFormat.VERSION_CURRENT);
numFields = readFields(in, state.fieldInfos);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(in, priorE);
}
}
this.numFields = numFields;
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
this.data = state.directory.openInput(dataName, state.context);
boolean success = false;

View File

@ -54,7 +54,6 @@ class Lucene49NormsProducer extends NormsProducer {
// metadata maps (just file pointers and minimal stuff)
private final Map<String,NormsEntry> norms = new HashMap<>();
private final IndexInput data;
private final int version;
// ram instances we have already loaded
final Map<String,NumericDocValues> instances = new HashMap<>();
@ -68,12 +67,19 @@ class Lucene49NormsProducer extends NormsProducer {
maxDoc = state.segmentInfo.getDocCount();
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
int version = -1;
// read in the entries from the metadata file.
try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
version = CodecUtil.checkHeader(in, metaCodec, VERSION_START, VERSION_CURRENT);
readFields(in, state.fieldInfos);
CodecUtil.checkFooter(in);
Throwable priorE = null;
try {
version = CodecUtil.checkHeader(in, metaCodec, VERSION_START, VERSION_CURRENT);
readFields(in, state.fieldInfos);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(in, priorE);
}
}
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);

View File

@ -0,0 +1,135 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 5.0 index format, with configurable per-field postings
* and docvalues formats.
* <p>
* If you want to reuse functionality of this codec in another codec, extend
* {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene50 package documentation for file format details.
* @lucene.experimental
*/
public class Lucene50Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return Lucene50Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Lucene50Codec.this.getDocValuesFormatForField(field);
}
};
/** Sole constructor. */
public Lucene50Codec() {
super("Lucene50");
}
@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 segmentInfosFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene41"
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultFormat;
}
/** Returns the docvalues format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene410"
*/
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("Lucene410");
private final NormsFormat normsFormat = new Lucene49NormsFormat();
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
}

View File

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

View File

@ -0,0 +1,127 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
/**
* Lucene 5.0 FieldInfos reader.
*
* @lucene.experimental
* @see Lucene50FieldInfosFormat
*/
final class Lucene50FieldInfosReader extends FieldInfosReader {
/** Sole constructor. */
public Lucene50FieldInfosReader() {
}
@Override
public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, Lucene50FieldInfosFormat.EXTENSION);
try (ChecksumIndexInput input = directory.openChecksumInput(fileName, context)) {
Throwable priorE = null;
FieldInfo infos[] = null;
try {
CodecUtil.checkHeader(input, Lucene50FieldInfosFormat.CODEC_NAME,
Lucene50FieldInfosFormat.FORMAT_START,
Lucene50FieldInfosFormat.FORMAT_CURRENT);
final int size = input.readVInt(); //read in the size
infos = new FieldInfo[size];
for (int i = 0; i < size; i++) {
String name = input.readString();
final int fieldNumber = input.readVInt();
if (fieldNumber < 0) {
throw new CorruptIndexException("invalid field number for field: " + name + ", fieldNumber=" + fieldNumber, input);
}
byte bits = input.readByte();
boolean isIndexed = (bits & Lucene50FieldInfosFormat.IS_INDEXED) != 0;
boolean storeTermVector = (bits & Lucene50FieldInfosFormat.STORE_TERMVECTOR) != 0;
boolean omitNorms = (bits & Lucene50FieldInfosFormat.OMIT_NORMS) != 0;
boolean storePayloads = (bits & Lucene50FieldInfosFormat.STORE_PAYLOADS) != 0;
final IndexOptions indexOptions;
if (!isIndexed) {
indexOptions = null;
} else if ((bits & Lucene50FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_ONLY;
} else if ((bits & Lucene50FieldInfosFormat.OMIT_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS;
} else if ((bits & Lucene50FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
} else {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
}
// DV Types are packed in one byte
byte val = input.readByte();
final DocValuesType docValuesType = getDocValuesType(input, (byte) (val & 0x0F));
final DocValuesType normsType = getDocValuesType(input, (byte) ((val >>> 4) & 0x0F));
final long dvGen = input.readLong();
final Map<String,String> attributes = input.readStringStringMap();
try {
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads,
indexOptions, docValuesType, normsType, dvGen, Collections.unmodifiableMap(attributes));
infos[i].checkConsistency();
} catch (IllegalStateException e) {
throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
}
}
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(input, priorE);
}
return new FieldInfos(infos);
}
}
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 if (b == 5) {
return DocValuesType.SORTED_NUMERIC;
} else {
throw new CorruptIndexException("invalid docvalues byte: " + b, input);
}
}
}

View File

@ -0,0 +1,103 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.IndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Lucene 5.0 FieldInfos writer.
*
* @see Lucene50FieldInfosFormat
* @lucene.experimental
*/
final class Lucene50FieldInfosWriter extends FieldInfosWriter {
/** Sole constructor. */
public Lucene50FieldInfosWriter() {
}
@Override
public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, Lucene50FieldInfosFormat.EXTENSION);
try (IndexOutput output = directory.createOutput(fileName, context)) {
CodecUtil.writeHeader(output, Lucene50FieldInfosFormat.CODEC_NAME, Lucene50FieldInfosFormat.FORMAT_CURRENT);
output.writeVInt(infos.size());
for (FieldInfo fi : infos) {
fi.checkConsistency();
IndexOptions indexOptions = fi.getIndexOptions();
byte bits = 0x0;
if (fi.hasVectors()) bits |= Lucene50FieldInfosFormat.STORE_TERMVECTOR;
if (fi.omitsNorms()) bits |= Lucene50FieldInfosFormat.OMIT_NORMS;
if (fi.hasPayloads()) bits |= Lucene50FieldInfosFormat.STORE_PAYLOADS;
if (fi.isIndexed()) {
bits |= Lucene50FieldInfosFormat.IS_INDEXED;
assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
if (indexOptions == IndexOptions.DOCS_ONLY) {
bits |= Lucene50FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS;
} else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
bits |= Lucene50FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS;
} else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
bits |= Lucene50FieldInfosFormat.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());
}
CodecUtil.writeFooter(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 if (type == DocValuesType.SORTED_NUMERIC) {
return 5;
} else {
throw new AssertionError();
}
}
}

View File

@ -0,0 +1,92 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.CodecUtil;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.codecs.SegmentInfoWriter;
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.SegmentInfo; // javadocs
import org.apache.lucene.index.SegmentInfos; // javadocs
import org.apache.lucene.store.DataOutput; // javadocs
/**
* Lucene 5.0 Segment info format.
* <p>
* Files:
* <ul>
* <li><tt>.si</tt>: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, Id, Footer
* </ul>
* </p>
* Data types:
* <p>
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
* <li>Files --&gt; {@link DataOutput#writeStringSet Set&lt;String&gt;}</li>
* <li>Diagnostics --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
* <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* <li>Id --&gt; {@link DataOutput#writeString String}</li>
* </ul>
* </p>
* Field Descriptions:
* <p>
* <ul>
* <li>SegVersion is the code version that created the segment.</li>
* <li>SegSize is the number of documents contained in the segment index.</li>
* <li>IsCompoundFile records whether the segment is written as a compound file or
* not. If this is -1, the segment is not a compound file. If it is 1, the segment
* is a compound file.</li>
* <li>The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid,
* for each segment it creates. It includes metadata like the current Lucene
* version, OS, Java version, why the segment was created (merge, flush,
* addIndexes), etc.</li>
* <li>Files is a list of files referred to by this segment.</li>
* </ul>
* </p>
*
* @see SegmentInfos
* @lucene.experimental
*/
public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
private final SegmentInfoReader reader = new Lucene50SegmentInfoReader();
private final SegmentInfoWriter writer = new Lucene50SegmentInfoWriter();
/** Sole constructor. */
public Lucene50SegmentInfoFormat() {
}
@Override
public SegmentInfoReader getSegmentInfoReader() {
return reader;
}
@Override
public SegmentInfoWriter getSegmentInfoWriter() {
return writer;
}
/** File extension used to store {@link SegmentInfo}. */
public final static String SI_EXTENSION = "si";
static final String CODEC_NAME = "Lucene50SegmentInfo";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
}

View File

@ -0,0 +1,84 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.text.ParseException;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Version;
/**
* Lucene 5.0 implementation of {@link SegmentInfoReader}.
*
* @see Lucene50SegmentInfoFormat
* @lucene.experimental
*/
public class Lucene50SegmentInfoReader extends SegmentInfoReader {
/** Sole constructor. */
public Lucene50SegmentInfoReader() {
}
@Override
public SegmentInfo read(Directory dir, String segment, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
Throwable priorE = null;
SegmentInfo si = null;
try {
CodecUtil.checkHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
Lucene50SegmentInfoFormat.VERSION_START,
Lucene50SegmentInfoFormat.VERSION_CURRENT);
final Version version;
try {
version = Version.parse(input.readString());
} catch (ParseException pe) {
throw new CorruptIndexException("unable to parse version string: " + pe.getMessage(), input, pe);
}
final int docCount = input.readInt();
if (docCount < 0) {
throw new CorruptIndexException("invalid docCount: " + docCount, input);
}
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final Map<String,String> diagnostics = input.readStringStringMap();
final Set<String> files = input.readStringSet();
String id = input.readString();
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, id);
si.setFiles(files);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(input, priorE);
}
return si;
}
}
}

View File

@ -0,0 +1,80 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.SegmentInfoWriter;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.Version;
/**
* Lucene 5.0 implementation of {@link SegmentInfoWriter}.
*
* @see Lucene50SegmentInfoFormat
* @lucene.experimental
*/
public class Lucene50SegmentInfoWriter extends SegmentInfoWriter {
/** Sole constructor. */
public Lucene50SegmentInfoWriter() {
}
/** Save a single segment's info. */
@Override
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
si.addFile(fileName);
final IndexOutput output = dir.createOutput(fileName, ioContext);
boolean success = false;
try {
CodecUtil.writeHeader(output, Lucene50SegmentInfoFormat.CODEC_NAME, Lucene50SegmentInfoFormat.VERSION_CURRENT);
Version version = si.getVersion();
if (version.major < 5) {
throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si);
}
// Write the Lucene version that created this segment, since 3.1
output.writeString(version.toString());
output.writeInt(si.getDocCount());
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
output.writeStringStringMap(si.getDiagnostics());
output.writeStringSet(si.files());
output.writeString(si.getId());
CodecUtil.writeFooter(output);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
// TODO: are we doing this outside of the tracking wrapper? why must SIWriter cleanup like this?
IOUtils.deleteFilesIgnoringExceptions(si.dir, fileName);
} else {
output.close();
}
}
}
}

View File

@ -0,0 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Lucene 5.0 file format.
</body>
</html>

View File

@ -170,21 +170,21 @@ public class CheckIndex {
/** Current deletions generation. */
public long deletionsGen;
/** Number of deleted documents. */
public int numDeleted;
/** True if we were able to open an AtomicReader on this
* segment. */
public boolean openReaderPassed;
/** Number of fields in this segment. */
int numFields;
/** Map that includes certain
* debugging details that IndexWriter records into
* each segment it creates */
public Map<String,String> diagnostics;
/** Status for testing of livedocs */
public LiveDocStatus liveDocStatus;
/** Status for testing of field infos */
public FieldInfoStatus fieldInfoStatus;
/** Status for testing of field norms (null if field norms could not be tested). */
public FieldNormStatus fieldNormStatus;
@ -201,6 +201,34 @@ public class CheckIndex {
public DocValuesStatus docValuesStatus;
}
/**
* Status from testing livedocs
*/
public static final class LiveDocStatus {
private LiveDocStatus() {
}
/** Number of deleted documents. */
public int numDeleted;
/** Exception thrown during term index test (null on success) */
public Throwable error = null;
}
/**
* Status from testing field infos.
*/
public static final class FieldInfoStatus {
private FieldInfoStatus() {
}
/** Number of fields successfully tested */
public long totFields = 0L;
/** Exception thrown during term index test (null on success) */
public Throwable error = null;
}
/**
* Status from testing field norms.
*/
@ -574,63 +602,34 @@ public class CheckIndex {
reader.checkIntegrity();
msg(infoStream, "OK");
if (infoStream != null)
infoStream.print(" test: check live docs.....");
final int numDocs = reader.numDocs();
toLoseDocCount = numDocs;
if (reader.hasDeletions()) {
if (reader.numDocs() != info.info.getDocCount() - info.getDelCount()) {
throw new RuntimeException("delete count mismatch: info=" + (info.info.getDocCount() - info.getDelCount()) + " vs reader=" + reader.numDocs());
}
if ((info.info.getDocCount()-reader.numDocs()) > reader.maxDoc()) {
throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.info.getDocCount()-reader.numDocs()));
}
if (info.info.getDocCount() - numDocs != info.getDelCount()) {
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
}
Bits liveDocs = reader.getLiveDocs();
if (liveDocs == null) {
throw new RuntimeException("segment should have deletions, but liveDocs is null");
} else {
int numLive = 0;
for (int j = 0; j < liveDocs.length(); j++) {
if (liveDocs.get(j)) {
numLive++;
}
}
if (numLive != numDocs) {
throw new RuntimeException("liveDocs count mismatch: info=" + numDocs + ", vs bits=" + numLive);
}
}
segInfoStat.numDeleted = info.info.getDocCount() - numDocs;
msg(infoStream, "OK [" + (segInfoStat.numDeleted) + " deleted docs]");
} else {
if (info.getDelCount() != 0) {
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
}
Bits liveDocs = reader.getLiveDocs();
if (liveDocs != null) {
// its ok for it to be non-null here, as long as none are set right?
for (int j = 0; j < liveDocs.length(); j++) {
if (!liveDocs.get(j)) {
throw new RuntimeException("liveDocs mismatch: info says no deletions but doc " + j + " is deleted.");
}
}
}
msg(infoStream, "OK");
}
if (reader.maxDoc() != info.info.getDocCount()) {
throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.info.getDocCount());
}
// Test getFieldInfos()
if (infoStream != null) {
infoStream.print(" test: fields..............");
final int numDocs = reader.numDocs();
toLoseDocCount = numDocs;
if (reader.hasDeletions()) {
if (reader.numDocs() != info.info.getDocCount() - info.getDelCount()) {
throw new RuntimeException("delete count mismatch: info=" + (info.info.getDocCount() - info.getDelCount()) + " vs reader=" + reader.numDocs());
}
if ((info.info.getDocCount() - reader.numDocs()) > reader.maxDoc()) {
throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.info.getDocCount() - reader.numDocs()));
}
if (info.info.getDocCount() - reader.numDocs() != info.getDelCount()) {
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - reader.numDocs()));
}
} else {
if (info.getDelCount() != 0) {
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - reader.numDocs()));
}
}
FieldInfos fieldInfos = reader.getFieldInfos();
msg(infoStream, "OK [" + fieldInfos.size() + " fields]");
segInfoStat.numFields = fieldInfos.size();
// Test Livedocs
segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
// Test Fieldinfos
segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
// Test Field Norms
segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
@ -648,7 +647,11 @@ public class CheckIndex {
// Rethrow the first exception we encountered
// This will cause stats for failed segments to be incremented properly
if (segInfoStat.fieldNormStatus.error != null) {
if (segInfoStat.liveDocStatus.error != null) {
throw new RuntimeException("Live docs test failed");
} else if (segInfoStat.fieldInfoStatus.error != null) {
throw new RuntimeException("Field Info test failed");
} else if (segInfoStat.fieldNormStatus.error != null) {
throw new RuntimeException("Field Norm test failed");
} else if (segInfoStat.termIndexStatus.error != null) {
throw new RuntimeException("Term Index test failed");
@ -708,6 +711,94 @@ public class CheckIndex {
return result;
}
/**
* Test live docs.
* @lucene.experimental
*/
public static Status.LiveDocStatus testLiveDocs(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
final Status.LiveDocStatus status = new Status.LiveDocStatus();
try {
if (infoStream != null)
infoStream.print(" test: check live docs.....");
final int numDocs = reader.numDocs();
if (reader.hasDeletions()) {
Bits liveDocs = reader.getLiveDocs();
if (liveDocs == null) {
throw new RuntimeException("segment should have deletions, but liveDocs is null");
} else {
int numLive = 0;
for (int j = 0; j < liveDocs.length(); j++) {
if (liveDocs.get(j)) {
numLive++;
}
}
if (numLive != numDocs) {
throw new RuntimeException("liveDocs count mismatch: info=" + numDocs + ", vs bits=" + numLive);
}
}
status.numDeleted = reader.numDeletedDocs();
msg(infoStream, "OK [" + (status.numDeleted) + " deleted docs]");
} else {
Bits liveDocs = reader.getLiveDocs();
if (liveDocs != null) {
// its ok for it to be non-null here, as long as none are set right?
for (int j = 0; j < liveDocs.length(); j++) {
if (!liveDocs.get(j)) {
throw new RuntimeException("liveDocs mismatch: info says no deletions but doc " + j + " is deleted.");
}
}
}
msg(infoStream, "OK");
}
} catch (Throwable e) {
if (failFast) {
IOUtils.reThrow(e);
}
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
}
}
return status;
}
/**
* Test field infos.
* @lucene.experimental
*/
public static Status.FieldInfoStatus testFieldInfos(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
final Status.FieldInfoStatus status = new Status.FieldInfoStatus();
try {
// Test Field Infos
if (infoStream != null) {
infoStream.print(" test: field infos.........");
}
FieldInfos fieldInfos = reader.getFieldInfos();
for (FieldInfo f : fieldInfos) {
f.checkConsistency();
}
msg(infoStream, "OK [" + fieldInfos.size() + " fields]");
status.totFields = fieldInfos.size();
} catch (Throwable e) {
if (failFast) {
IOUtils.reThrow(e);
}
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
}
}
return status;
}
/**
* Test field norms.
* @lucene.experimental

View File

@ -149,24 +149,45 @@ public final class FieldInfo {
assert checkConsistency();
}
private boolean checkConsistency() {
if (!indexed) {
assert !storeTermVector;
assert !storePayloads;
assert !omitNorms;
assert normType == null;
assert indexOptions == null;
} else {
assert indexOptions != null;
/**
* Performs internal consistency checks.
* Always returns true (or throws IllegalStateException)
*/
public boolean checkConsistency() {
if (indexed) {
if (indexOptions == null) {
throw new IllegalStateException("indexed field '" + name + "' must have index options");
}
if (omitNorms) {
assert normType == null;
if (normType != null) {
throw new IllegalStateException("indexed field '" + name + "' cannot both omit norms and have norms");
}
}
// Cannot store payloads unless positions are indexed:
assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !this.storePayloads;
if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0 && storePayloads) {
throw new IllegalStateException("indexed field '" + name + "' cannot have payloads without positions");
}
} else {
if (storeTermVector) {
throw new IllegalStateException("non-indexed field '" + name + "' cannot store term vectors");
}
if (storePayloads) {
throw new IllegalStateException("non-indexed field '" + name + "' cannot store payloads");
}
if (omitNorms) {
throw new IllegalStateException("non-indexed field '" + name + "' cannot omit norms");
}
if (normType != null) {
throw new IllegalStateException("non-indexed field '" + name + "' cannot have norms");
}
if (indexOptions != null) {
throw new IllegalStateException("non-indexed field '" + name + "' cannot have index options");
}
}
if (dvGen != -1) {
assert docValueType != null;
if (dvGen != -1 && docValueType == null) {
throw new IllegalStateException("field '" + name + "' cannot have a docvalues update generation without having docvalues");
}
return true;

View File

@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene410.Lucene410Codec
org.apache.lucene.codecs.lucene50.Lucene50Codec

View File

@ -0,0 +1,197 @@
package org.apache.lucene.index;
/*
* 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.CodecUtil;
import org.apache.lucene.store.BufferedChecksumIndexInput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMFile;
import org.apache.lucene.store.RAMInputStream;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.LuceneTestCase;
/** tests for codecutil methods */
public class TestCodecUtil extends LuceneTestCase {
public void testHeaderLength() throws Exception {
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
output.close();
IndexInput input = new RAMInputStream("file", file);
input.seek(CodecUtil.headerLength("FooBar"));
assertEquals("this is the data", input.readString());
input.close();
}
public void testWriteTooLongHeader() throws Exception {
StringBuilder tooLong = new StringBuilder();
for (int i = 0; i < 128; i++) {
tooLong.append('a');
}
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
try {
CodecUtil.writeHeader(output, tooLong.toString(), 5);
fail("didn't get expected exception");
} catch (IllegalArgumentException expected) {
// expected
}
}
public void testWriteNonAsciiHeader() throws Exception {
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
try {
CodecUtil.writeHeader(output, "\u1234", 5);
fail("didn't get expected exception");
} catch (IllegalArgumentException expected) {
// expected
}
}
public void testReadHeaderWrongMagic() throws Exception {
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
output.writeInt(1234);
output.close();
IndexInput input = new RAMInputStream("file", file);
try {
CodecUtil.checkHeader(input, "bogus", 1, 1);
fail("didn't get expected exception");
} catch (CorruptIndexException expected) {
// expected
}
}
public void testChecksumEntireFile() throws Exception {
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
CodecUtil.writeFooter(output);
output.close();
IndexInput input = new RAMInputStream("file", file);
CodecUtil.checksumEntireFile(input);
input.close();
}
public void testCheckFooterValid() throws Exception {
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
CodecUtil.writeFooter(output);
output.close();
ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
Exception mine = new RuntimeException("fake exception");
try {
CodecUtil.checkFooter(input, mine);
fail("didn't get expected exception");
} catch (RuntimeException expected) {
assertEquals("fake exception", expected.getMessage());
Throwable suppressed[] = expected.getSuppressed();
assertEquals(1, suppressed.length);
assertTrue(suppressed[0].getMessage().contains("checksum passed"));
}
input.close();
}
public void testCheckFooterValidAtFooter() throws Exception {
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
CodecUtil.writeFooter(output);
output.close();
ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
CodecUtil.checkHeader(input, "FooBar", 5, 5);
assertEquals("this is the data", input.readString());
Exception mine = new RuntimeException("fake exception");
try {
CodecUtil.checkFooter(input, mine);
fail("didn't get expected exception");
} catch (RuntimeException expected) {
assertEquals("fake exception", expected.getMessage());
Throwable suppressed[] = expected.getSuppressed();
assertEquals(1, suppressed.length);
assertTrue(suppressed[0].getMessage().contains("checksum passed"));
}
input.close();
}
public void testCheckFooterValidPastFooter() throws Exception {
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
CodecUtil.writeFooter(output);
output.close();
ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
CodecUtil.checkHeader(input, "FooBar", 5, 5);
assertEquals("this is the data", input.readString());
// bogusly read a byte too far (can happen)
input.readByte();
Exception mine = new RuntimeException("fake exception");
try {
CodecUtil.checkFooter(input, mine);
fail("didn't get expected exception");
} catch (RuntimeException expected) {
assertEquals("fake exception", expected.getMessage());
Throwable suppressed[] = expected.getSuppressed();
assertEquals(1, suppressed.length);
assertTrue(suppressed[0].getMessage().contains("checksum status indeterminate"));
}
input.close();
}
public void testCheckFooterInvalid() throws Exception {
RAMFile file = new RAMFile();
IndexOutput output = new RAMOutputStream(file, true);
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
output.writeInt(CodecUtil.FOOTER_MAGIC);
output.writeInt(0);
output.writeLong(1234567); // write a bogus checksum
output.close();
ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
CodecUtil.checkHeader(input, "FooBar", 5, 5);
assertEquals("this is the data", input.readString());
Exception mine = new RuntimeException("fake exception");
try {
CodecUtil.checkFooter(input, mine);
fail("didn't get expected exception");
} catch (RuntimeException expected) {
assertEquals("fake exception", expected.getMessage());
Throwable suppressed[] = expected.getSuppressed();
assertEquals(1, suppressed.length);
assertTrue(suppressed[0].getMessage().contains("checksum failed"));
}
input.close();
}
}

View File

@ -46,8 +46,8 @@ import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
import org.apache.lucene.codecs.lucene50.Lucene50Codec;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@ -64,11 +64,6 @@ import org.apache.lucene.index.FilterLeafReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
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.ConcurrentMergeScheduler;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
@ -248,11 +243,13 @@ public final class TestUtil {
PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
reader.checkIntegrity();
FieldNormStatus fieldNormStatus = CheckIndex.testFieldNorms(reader, infoStream, true);
TermIndexStatus termIndexStatus = CheckIndex.testPostings(reader, infoStream, false, true);
StoredFieldStatus storedFieldStatus = CheckIndex.testStoredFields(reader, infoStream, true);
TermVectorStatus termVectorStatus = CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors, true);
DocValuesStatus docValuesStatus = CheckIndex.testDocValues(reader, infoStream, true);
CheckIndex.testLiveDocs(reader, infoStream, true);
CheckIndex.testFieldInfos(reader, infoStream, true);
CheckIndex.testFieldNorms(reader, infoStream, true);
CheckIndex.testPostings(reader, infoStream, false, true);
CheckIndex.testStoredFields(reader, infoStream, true);
CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors, true);
CheckIndex.testDocValues(reader, infoStream, true);
if (LuceneTestCase.INFOSTREAM) {
System.out.println(bos.toString(IOUtils.UTF_8));
@ -737,7 +734,7 @@ public final class TestUtil {
* This may be different than {@link Codec#getDefault()} because that is randomized.
*/
public static Codec getDefaultCodec() {
return new Lucene410Codec();
return new Lucene50Codec();
}
/**

View File

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