From 6384ae9fb70a314913aaec1ecd7ffb66d1adf1f5 Mon Sep 17 00:00:00 2001 From: Robert Muir Date: Tue, 10 Jun 2014 11:35:48 +0000 Subject: [PATCH] LUCENE-5743: Add Lucene49NormsFormat git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1601606 13f79535-47bb-0310-9956-ffa450edef68 --- lucene/CHANGES.txt | 3 + .../codecs/diskdv/TestDiskNormsFormat.java | 32 +++ .../simpletext/TestSimpleTextNormsFormat.java | 31 +++ .../lucene/codecs/lucene49/Lucene49Codec.java | 2 +- .../lucene49/Lucene49DocValuesConsumer.java | 2 + .../lucene49/Lucene49NormsConsumer.java | 208 ++++++++++++++++ .../codecs/lucene49/Lucene49NormsFormat.java | 121 +++++++++ .../lucene49/Lucene49NormsProducer.java | 233 ++++++++++++++++++ .../lucene/codecs/lucene49/package.html | 4 +- .../lucene40/TestLucene40NormsFormat.java | 38 +++ .../lucene42/TestLucene42NormsFormat.java | 38 +++ .../lucene49/TestLucene49NormsFormat.java | 33 +++ .../apache/lucene/index/TestNormsFormat.java | 30 +++ .../lucene/index/BaseNormsFormatTestCase.java | 183 ++++++++++++++ 14 files changed, 955 insertions(+), 3 deletions(-) create mode 100644 lucene/codecs/src/test/org/apache/lucene/codecs/diskdv/TestDiskNormsFormat.java create mode 100644 lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextNormsFormat.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40NormsFormat.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42NormsFormat.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene49/TestLucene49NormsFormat.java create mode 100644 lucene/core/src/test/org/apache/lucene/index/TestNormsFormat.java create mode 100644 lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 9cdcae44a72..253465bc981 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -129,6 +129,9 @@ New Features from Directory. Add Lucene49Codec and Lucene49DocValuesFormat that make use of these. (Robert Muir) +* LUCENE-5743: Add Lucene49NormsFormat, which can compress in some cases + such as very short fields. (Ryan Ernst, Adrien Grand, Robert Muir) + Changes in Backwards Compatibility Policy * LUCENE-5634: Add reuse argument to IndexableField.tokenStream. This diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/diskdv/TestDiskNormsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/diskdv/TestDiskNormsFormat.java new file mode 100644 index 00000000000..9df5d162f0c --- /dev/null +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/diskdv/TestDiskNormsFormat.java @@ -0,0 +1,32 @@ +package org.apache.lucene.codecs.diskdv; + +/* + * 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.cheapbastard.CheapBastardCodec; +import org.apache.lucene.index.BaseNormsFormatTestCase; + +/** Tests DiskNormsFormat */ +public class TestDiskNormsFormat extends BaseNormsFormatTestCase { + private final Codec codec = new CheapBastardCodec(); + + @Override + protected Codec getCodec() { + return codec; + } +} diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextNormsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextNormsFormat.java new file mode 100644 index 00000000000..0a88ca802da --- /dev/null +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextNormsFormat.java @@ -0,0 +1,31 @@ +package org.apache.lucene.codecs.simpletext; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.BaseNormsFormatTestCase; + +/** Tests SimpleTextNormsFormat */ +public class TestSimpleTextNormsFormat extends BaseNormsFormatTestCase { + private final Codec codec = new SimpleTextCodec(); + + @Override + protected Codec getCodec() { + return codec; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java index 4d7d344ef81..1b64b9b68b9 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java @@ -131,7 +131,7 @@ public class Lucene49Codec extends Codec { private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41"); private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene49"); - private final NormsFormat normsFormat = new Lucene42NormsFormat(); + private final NormsFormat normsFormat = new Lucene49NormsFormat(); @Override public final NormsFormat normsFormat() { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java index f974344436b..b2a8b0edb02 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java @@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene49; import java.io.Closeable; // javadocs import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -199,6 +200,7 @@ public class Lucene49DocValuesConsumer extends DocValuesConsumer implements Clos break; case TABLE_COMPRESSED: final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]); + Arrays.sort(decode); final HashMap encode = new HashMap<>(); meta.writeVInt(decode.length); for (int i = 0; i < decode.length; i++) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java new file mode 100644 index 00000000000..fc81b3c3205 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java @@ -0,0 +1,208 @@ +package org.apache.lucene.codecs.lucene49; + +/* + * 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.Arrays; +import java.util.HashMap; +import java.util.HashSet; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.packed.BlockPackedWriter; +import org.apache.lucene.util.packed.PackedInts; + +import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_CURRENT; + +/** + * Writer for {@link Lucene49NormsFormat} + */ +class Lucene49NormsConsumer extends DocValuesConsumer { + static final byte DELTA_COMPRESSED = 0; + static final byte TABLE_COMPRESSED = 1; + static final byte CONST_COMPRESSED = 2; + static final byte UNCOMPRESSED = 3; + static final int BLOCK_SIZE = 16384; + + IndexOutput data, meta; + final int maxDoc; + + Lucene49NormsConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + maxDoc = state.segmentInfo.getDocCount(); + boolean success = false; + try { + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.createOutput(dataName, state.context); + CodecUtil.writeHeader(data, dataCodec, VERSION_CURRENT); + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + meta = state.directory.createOutput(metaName, state.context); + CodecUtil.writeHeader(meta, metaCodec, VERSION_CURRENT); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + // we explicitly use only certain bits per value and a specified format, so we statically check this will work + static { + assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(1); + assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(2); + assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(4); + } + + @Override + public void addNumericField(FieldInfo field, Iterable values) throws IOException { + meta.writeVInt(field.number); + long minValue = Long.MAX_VALUE; + long maxValue = Long.MIN_VALUE; + // TODO: more efficient? + HashSet uniqueValues = null; + uniqueValues = new HashSet<>(); + + long count = 0; + for (Number nv : values) { + if (nv == null) { + throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count); + } + final long v = nv.longValue(); + + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + + if (uniqueValues != null) { + if (uniqueValues.add(v)) { + if (uniqueValues.size() > 256) { + uniqueValues = null; + } + } + } + ++count; + } + + if (count != maxDoc) { + throw new IllegalStateException("illegal norms data for field " + field.name + ", expected " + maxDoc + " values, got " + count); + } + + if (uniqueValues != null && uniqueValues.size() == 1) { + // 0 bpv + meta.writeByte(CONST_COMPRESSED); + meta.writeLong(minValue); + } else if (uniqueValues != null) { + // small number of unique values: this is the typical case: + // we only use bpv=1,2,4,8 + PackedInts.Format format = PackedInts.Format.PACKED_SINGLE_BLOCK; + int bitsPerValue = PackedInts.bitsRequired(uniqueValues.size()-1); + if (bitsPerValue == 3) { + bitsPerValue = 4; + } else if (bitsPerValue > 4) { + bitsPerValue = 8; + } + + if (bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) { + meta.writeByte(UNCOMPRESSED); // uncompressed byte[] + meta.writeLong(data.getFilePointer()); + for (Number nv : values) { + data.writeByte(nv == null ? 0 : (byte) nv.longValue()); + } + } else { + meta.writeByte(TABLE_COMPRESSED); // table-compressed + meta.writeLong(data.getFilePointer()); + data.writeVInt(PackedInts.VERSION_CURRENT); + + Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]); + Arrays.sort(decode); + final HashMap encode = new HashMap<>(); + // upgrade to power of two sized array + int size = 1 << bitsPerValue; + data.writeVInt(size); + for (int i = 0; i < decode.length; i++) { + data.writeLong(decode[i]); + encode.put(decode[i], i); + } + for (int i = decode.length; i < size; i++) { + data.writeLong(0); + } + + data.writeVInt(format.getId()); + data.writeVInt(bitsPerValue); + + final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, format, maxDoc, bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE); + for(Number nv : values) { + writer.add(encode.get(nv.longValue())); + } + writer.finish(); + } + } else { + meta.writeByte(DELTA_COMPRESSED); // delta-compressed + meta.writeLong(data.getFilePointer()); + data.writeVInt(PackedInts.VERSION_CURRENT); + data.writeVInt(BLOCK_SIZE); + + final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE); + for (Number nv : values) { + writer.add(nv.longValue()); + } + writer.finish(); + } + } + + @Override + public void close() throws IOException { + boolean success = false; + try { + if (meta != null) { + meta.writeVInt(-1); // write EOF marker + CodecUtil.writeFooter(meta); // write checksum + } + if (data != null) { + CodecUtil.writeFooter(data); // write checksum + } + success = true; + } finally { + if (success) { + IOUtils.close(data, meta); + } else { + IOUtils.closeWhileHandlingException(data, meta); + } + meta = data = null; + } + } + + @Override + public void addBinaryField(FieldInfo field, final Iterable values) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void addSortedField(FieldInfo field, Iterable values, Iterable docToOrd) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void addSortedSetField(FieldInfo field, Iterable values, final Iterable docToOrdCount, final Iterable ords) throws IOException { + throw new UnsupportedOperationException(); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java new file mode 100644 index 00000000000..9975b0d88b8 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java @@ -0,0 +1,121 @@ +package org.apache.lucene.codecs.lucene49; + +/* + * 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.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.SmallFloat; +import org.apache.lucene.util.packed.BlockPackedWriter; +import org.apache.lucene.util.packed.PackedInts; + +/** + * Lucene 4.9 Score normalization format. + *

+ * Encodes normalization values with these strategies: + *

+ *

    + *
  • Uncompressed: when values fit into a single byte and would require more than 4 bits + * per value, they are just encoded as an uncompressed byte array. + *
  • Constant: when there is only one value present for the entire field, no actual data + * is written: this constant is encoded in the metadata + *
  • Table-compressed: when the number of unique values is very small (< 64), and + * when there are unused "gaps" in the range of values used (such as {@link SmallFloat}), + * a lookup table is written instead. Each per-document entry is instead the ordinal + * to this table, and those ordinals are compressed with bitpacking ({@link PackedInts}). + *
  • Delta-compressed: per-document integers written as deltas from the minimum value, + * compressed with bitpacking. For more information, see {@link BlockPackedWriter}. + * This is only used when norms of larger than one byte are present. + *
+ *

+ * Files: + *

    + *
  1. .nvd: Norms data
  2. + *
  3. .nvm: Norms metadata
  4. + *
+ *
    + *
  1. + *

    The Norms metadata or .nvm file.

    + *

    For each norms field, this stores metadata, such as the offset into the + * Norms data (.nvd)

    + *

    Norms metadata (.dvm) --> Header,<Entry>NumFields,Footer

    + *
      + *
    • Header --> {@link CodecUtil#writeHeader CodecHeader}
    • + *
    • Entry --> FieldNumber,Type,Offset
    • + *
    • FieldNumber --> {@link DataOutput#writeVInt vInt}
    • + *
    • Type --> {@link DataOutput#writeByte Byte}
    • + *
    • Offset --> {@link DataOutput#writeLong Int64}
    • + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter}
    • + *
    + *

    FieldNumber of -1 indicates the end of metadata.

    + *

    Offset is the pointer to the start of the data in the norms data (.nvd), or the singleton value for Constant

    + *

    Type indicates how Numeric values will be compressed: + *

      + *
    • 0 --> delta-compressed. For each block of 16k integers, every integer is delta-encoded + * from the minimum value within the block. + *
    • 1 --> table-compressed. When the number of unique numeric values is small and it would save space, + * a lookup table of unique values is written, followed by the ordinal for each document. + *
    • 2 --> constant. When there is a single value for the entire field. + *
    • 3 --> uncompressed: Values written as a simple byte[]. + *
    + *
  2. + *

    The Norms data or .nvd file.

    + *

    For each Norms field, this stores the actual per-document data (the heavy-lifting)

    + *

    Norms data (.nvd) --> Header,<Uncompressed | TableCompressed | DeltaCompressed>NumFields,Footer

    + *
      + *
    • Header --> {@link CodecUtil#writeHeader CodecHeader}
    • + *
    • Uncompressed --> {@link DataOutput#writeByte Byte}maxDoc
    • + *
    • TableCompressed --> PackedIntsVersion,Table,BitPackedData
    • + *
    • Table --> TableSize, {@link DataOutput#writeLong int64}TableSize
    • + *
    • BitpackedData --> {@link PackedInts}
    • + *
    • DeltaCompressed --> PackedIntsVersion,BlockSize,DeltaCompressedData
    • + *
    • DeltaCompressedData --> {@link BlockPackedWriter BlockPackedWriter(blockSize=16k)}
    • + *
    • PackedIntsVersion,BlockSize,TableSize --> {@link DataOutput#writeVInt vInt}
    • + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter}
    • + *
    + *
+ * @lucene.experimental + */ +public class Lucene49NormsFormat extends NormsFormat { + + /** Sole Constructor */ + public Lucene49NormsFormat() {} + + @Override + public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException { + return new Lucene49NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + } + + @Override + public DocValuesProducer normsProducer(SegmentReadState state) throws IOException { + return new Lucene49NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + } + + private static final String DATA_CODEC = "Lucene49NormsData"; + private static final String DATA_EXTENSION = "nvd"; + private static final String METADATA_CODEC = "Lucene49NormsMetadata"; + private static final String METADATA_EXTENSION = "nvm"; + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java new file mode 100644 index 00000000000..b3b4781bed6 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java @@ -0,0 +1,233 @@ +package org.apache.lucene.codecs.lucene49; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BinaryDocValues; +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.NumericDocValues; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.packed.BlockPackedReader; +import org.apache.lucene.util.packed.PackedInts; + +import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_START; +import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_CURRENT; +import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.CONST_COMPRESSED; +import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.DELTA_COMPRESSED; +import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.TABLE_COMPRESSED; +import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.UNCOMPRESSED; + +/** + * Reader for {@link Lucene49NormsFormat} + */ +class Lucene49NormsProducer extends DocValuesProducer { + // metadata maps (just file pointers and minimal stuff) + private final Map norms = new HashMap<>(); + private final IndexInput data; + private final int version; + + // ram instances we have already loaded + final Map instances = new HashMap<>(); + + private final int maxDoc; + private final AtomicLong ramBytesUsed; + + Lucene49NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + maxDoc = state.segmentInfo.getDocCount(); + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + // read in the entries from the metadata file. + ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context); + boolean success = false; + ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass())); + try { + version = CodecUtil.checkHeader(in, metaCodec, VERSION_START, VERSION_CURRENT); + readFields(in, state.fieldInfos); + CodecUtil.checkFooter(in); + success = true; + } finally { + if (success) { + IOUtils.close(in); + } else { + IOUtils.closeWhileHandlingException(in); + } + } + + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + this.data = state.directory.openInput(dataName, state.context); + success = false; + try { + final int version2 = CodecUtil.checkHeader(data, dataCodec, VERSION_START, VERSION_CURRENT); + if (version != version2) { + throw new CorruptIndexException("Format versions mismatch"); + } + + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this.data); + } + } + } + + private void readFields(IndexInput meta, FieldInfos infos) throws IOException { + int fieldNumber = meta.readVInt(); + while (fieldNumber != -1) { + FieldInfo info = infos.fieldInfo(fieldNumber); + if (info == null) { + throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")"); + } else if (!info.hasNorms()) { + throw new CorruptIndexException("Invalid field: " + info.name + " (resource=" + meta + ")"); + } + NormsEntry entry = new NormsEntry(); + entry.format = meta.readByte(); + entry.offset = meta.readLong(); + switch(entry.format) { + case CONST_COMPRESSED: + case UNCOMPRESSED: + case TABLE_COMPRESSED: + case DELTA_COMPRESSED: + break; + default: + throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta); + } + norms.put(fieldNumber, entry); + fieldNumber = meta.readVInt(); + } + } + + @Override + public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException { + NumericDocValues instance = instances.get(field.number); + if (instance == null) { + instance = loadNorms(field); + instances.put(field.number, instance); + } + return instance; + } + + @Override + public long ramBytesUsed() { + return ramBytesUsed.get(); + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(data); + } + + private NumericDocValues loadNorms(FieldInfo field) throws IOException { + NormsEntry entry = norms.get(field.number); + switch(entry.format) { + case CONST_COMPRESSED: + final long v = entry.offset; + return new NumericDocValues() { + @Override + public long get(int docID) { + return v; + } + }; + case UNCOMPRESSED: + data.seek(entry.offset); + final byte bytes[] = new byte[maxDoc]; + data.readBytes(bytes, 0, bytes.length); + ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes)); + return new NumericDocValues() { + @Override + public long get(int docID) { + return bytes[docID]; + } + }; + case DELTA_COMPRESSED: + data.seek(entry.offset); + int packedIntsVersion = data.readVInt(); + int blockSize = data.readVInt(); + final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, maxDoc, false); + ramBytesUsed.addAndGet(reader.ramBytesUsed()); + return reader; + case TABLE_COMPRESSED: + data.seek(entry.offset); + int packedVersion = data.readVInt(); + int size = data.readVInt(); + if (size > 256) { + throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + data); + } + final long decode[] = new long[size]; + for (int i = 0; i < decode.length; i++) { + decode[i] = data.readLong(); + } + final int formatID = data.readVInt(); + final int bitsPerValue = data.readVInt(); + final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedVersion, maxDoc, bitsPerValue); + ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed()); + return new NumericDocValues() { + @Override + public long get(int docID) { + return decode[(int)ordsReader.get(docID)]; + } + }; + default: + throw new AssertionError(); + } + } + + @Override + public BinaryDocValues getBinary(FieldInfo field) throws IOException { + throw new IllegalStateException(); + } + + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + throw new IllegalStateException(); + } + + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + throw new IllegalStateException(); + } + + @Override + public Bits getDocsWithField(FieldInfo field) throws IOException { + throw new IllegalStateException(); + } + + @Override + public void close() throws IOException { + data.close(); + } + + static class NormsEntry { + byte format; + long offset; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html index 21c31d47801..bbf7101ff9a 100755 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html @@ -173,7 +173,7 @@ 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}. +{@link org.apache.lucene.codecs.lucene49.Lucene49NormsFormat Normalization factors}. For each field in each document, a value is stored that is multiplied into the score for hits on that field.
  • @@ -289,7 +289,7 @@ systems that frequently run out of file handles. Stores additional per-position metadata information such as character offsets and user payloads -{@link org.apache.lucene.codecs.lucene42.Lucene42NormsFormat Norms} +{@link org.apache.lucene.codecs.lucene49.Lucene49NormsFormat Norms} .nvd, .nvm Encodes length and boost factors for docs and fields diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40NormsFormat.java new file mode 100644 index 00000000000..f3fb65afc70 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40NormsFormat.java @@ -0,0 +1,38 @@ +package org.apache.lucene.codecs.lucene40; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.BaseNormsFormatTestCase; +import org.junit.BeforeClass; + + +/** Tests Lucene40's norms format */ +public class TestLucene40NormsFormat extends BaseNormsFormatTestCase { + final Codec codec = new Lucene40RWCodec(); + + @Override + protected Codec getCodec() { + return codec; + } + + @BeforeClass + public static void beforeClass() { + OLD_FORMAT_IMPERSONATION_IS_ACTIVE = true; // explicitly instantiates ancient codec + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42NormsFormat.java new file mode 100644 index 00000000000..9ee7827127a --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42NormsFormat.java @@ -0,0 +1,38 @@ +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 org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.BaseNormsFormatTestCase; +import org.junit.BeforeClass; + + +/** Tests Lucene42's norms format */ +public class TestLucene42NormsFormat extends BaseNormsFormatTestCase { + final Codec codec = new Lucene42RWCodec(); + + @Override + protected Codec getCodec() { + return codec; + } + + @BeforeClass + public static void beforeClass() { + OLD_FORMAT_IMPERSONATION_IS_ACTIVE = true; // explicitly instantiates ancient codec + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene49/TestLucene49NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene49/TestLucene49NormsFormat.java new file mode 100644 index 00000000000..aa744ee2727 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene49/TestLucene49NormsFormat.java @@ -0,0 +1,33 @@ +package org.apache.lucene.codecs.lucene49; + +/* + * 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.index.BaseNormsFormatTestCase; + +/** + * Tests Lucene49NormsFormat + */ +public class TestLucene49NormsFormat extends BaseNormsFormatTestCase { + final Codec codec = new Lucene49Codec(); + + @Override + protected Codec getCodec() { + return codec; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNormsFormat.java b/lucene/core/src/test/org/apache/lucene/index/TestNormsFormat.java new file mode 100644 index 00000000000..a85a5baef35 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/index/TestNormsFormat.java @@ -0,0 +1,30 @@ +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.Codec; + +/** Tests the codec configuration defined by LuceneTestCase randomly + */ +public class TestNormsFormat extends BaseNormsFormatTestCase { + + @Override + protected Codec getCodec() { + return Codec.getDefault(); + } +} diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java new file mode 100644 index 00000000000..95e6db2edb1 --- /dev/null +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java @@ -0,0 +1,183 @@ +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 java.io.IOException; +import java.util.Random; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.analysis.MockTokenizer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.StringField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.search.CollectionStatistics; +import org.apache.lucene.search.TermStatistics; +import org.apache.lucene.search.similarities.Similarity; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.TestUtil; + +/** + * Abstract class to do basic tests for a norms format. + * NOTE: This test focuses on the norms impl, nothing else. + * The [stretch] goal is for this test to be + * so thorough in testing a new NormsFormat that if this + * test passes, then all Lucene/Solr tests should also pass. Ie, + * if there is some bug in a given NormsFormat that this + * test fails to catch then this test needs to be improved! */ +public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCase { + + public void testByteRange() throws Exception { + int iterations = atLeast(1); + final Random r = random(); + for (int i = 0; i < iterations; i++) { + doTestNormsVersusStoredFields(new LongProducer() { + @Override + long next() { + return TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE); + } + }); + } + } + + public void testLongRange() throws Exception { + int iterations = atLeast(1); + final Random r = random(); + for (int i = 0; i < iterations; i++) { + doTestNormsVersusStoredFields(new LongProducer() { + @Override + long next() { + return TestUtil.nextLong(r, Long.MIN_VALUE, Long.MAX_VALUE); + } + }); + } + } + + public void testFewValues() throws Exception { + int iterations = atLeast(1); + final Random r = random(); + for (int i = 0; i < iterations; i++) { + doTestNormsVersusStoredFields(new LongProducer() { + @Override + long next() { + return r.nextBoolean() ? 20 : 3; + } + }); + } + } + + public void testAllZeros() throws Exception { + int iterations = atLeast(1); + final Random r = random(); + for (int i = 0; i < iterations; i++) { + doTestNormsVersusStoredFields(new LongProducer() { + @Override + long next() { + return 0; + } + }); + } + } + + private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception { + int numDocs = atLeast(500); + long norms[] = new long[numDocs]; + for (int i = 0; i < numDocs; i++) { + norms[i] = longs.next(); + } + + Directory dir = newDirectory(); + Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false); + IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer); + conf.setSimilarity(new CannedNormSimilarity(norms)); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf); + Document doc = new Document(); + Field idField = new StringField("id", "", Field.Store.NO); + Field storedField = newTextField("stored", "", Field.Store.YES); + doc.add(idField); + doc.add(storedField); + + for (int i = 0; i < numDocs; i++) { + idField.setStringValue(Integer.toString(i)); + long value = norms[i]; + storedField.setStringValue(Long.toString(value)); + writer.addDocument(doc); + if (random().nextInt(31) == 0) { + writer.commit(); + } + } + + // delete some docs + int numDeletions = random().nextInt(numDocs/10); + for (int i = 0; i < numDeletions; i++) { + int id = random().nextInt(numDocs); + writer.deleteDocuments(new Term("id", Integer.toString(id))); + } + + writer.shutdown(); + + // compare + DirectoryReader ir = DirectoryReader.open(dir); + for (AtomicReaderContext context : ir.leaves()) { + AtomicReader r = context.reader(); + NumericDocValues docValues = r.getNormValues("stored"); + for (int i = 0; i < r.maxDoc(); i++) { + long storedValue = Long.parseLong(r.document(i).get("stored")); + assertEquals(storedValue, docValues.get(i)); + } + } + ir.close(); + dir.close(); + } + + + static abstract class LongProducer { + abstract long next(); + } + + static class CannedNormSimilarity extends Similarity { + final long norms[]; + int index = 0; + + CannedNormSimilarity(long norms[]) { + this.norms = norms; + } + + @Override + public long computeNorm(FieldInvertState state) { + return norms[index++]; + } + + @Override + public SimWeight computeWeight(float queryBoost, CollectionStatistics collectionStats, TermStatistics... termStats) { + throw new UnsupportedOperationException(); + } + + @Override + public SimScorer simScorer(SimWeight weight, AtomicReaderContext context) throws IOException { + throw new UnsupportedOperationException(); + } + } + + @Override + protected void addRandomFields(Document doc) { + // TODO: improve + doc.add(new TextField("foobar", "boo", Field.Store.NO)); + } +}