You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/09/30 11:37:52 UTC
svn commit: r1628382 [2/2] - in /lucene/dev/branches/lucene5969/lucene:
backward-codecs/src/java/org/apache/lucene/codecs/lucene40/
backward-codecs/src/java/org/apache/lucene/codecs/lucene42/
backward-codecs/src/java/org/apache/lucene/codecs/lucene45/ ...
Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java?rev=1628382&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java Tue Sep 30 09:37:52 2014
@@ -0,0 +1,319 @@
+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.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NormsConsumer;
+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.FilterIterator;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
+
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
+
+/**
+ * Writer for {@link Lucene50NormsFormat}
+ */
+class Lucene50NormsConsumer extends NormsConsumer {
+ 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 byte INDIRECT = 4;
+ static final int BLOCK_SIZE = 1 << 14;
+
+ // threshold for indirect encoding, computed as 1 - 1/log2(maxint)
+ // norms are only read for matching postings... so this is the threshold
+ // where n log n operations < maxdoc (e.g. it performs similar to other fields)
+ static final float INDIRECT_THRESHOLD = 1 - 1 / 31F;
+
+ IndexOutput data, meta;
+
+ Lucene50NormsConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+ boolean success = false;
+ try {
+ String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+ data = state.directory.createOutput(dataName, state.context);
+ CodecUtil.writeSegmentHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId());
+ String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+ meta = state.directory.createOutput(metaName, state.context);
+ CodecUtil.writeSegmentHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId());
+ 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 addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
+ meta.writeVInt(field.number);
+ long minValue = Long.MAX_VALUE;
+ long maxValue = Long.MIN_VALUE;
+ // TODO: more efficient?
+ NormMap uniqueValues = new NormMap();
+
+ int count = 0;
+ int missingCount = 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();
+ if (v == 0) {
+ missingCount++;
+ }
+
+ 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 (uniqueValues != null && uniqueValues.size == 1) {
+ // 0 bpv
+ addConstant(minValue);
+ } else if (count > 256 && missingCount > count * INDIRECT_THRESHOLD) {
+ // sparse encoding
+ addIndirect(field, values, count, missingCount);
+ } else if (uniqueValues != null) {
+ // small number of unique values: this is the typical case:
+ FormatAndBits compression = fastestFormatAndBits(uniqueValues.size-1);
+
+ if (compression.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
+ addUncompressed(values, count);
+ } else {
+ addTableCompressed(values, compression, count, uniqueValues);
+ }
+ } else {
+ addDeltaCompressed(values, count);
+ }
+ }
+
+ private FormatAndBits fastestFormatAndBits(int max) {
+ // we only use bpv=1,2,4,8
+ PackedInts.Format format = PackedInts.Format.PACKED_SINGLE_BLOCK;
+ int bitsPerValue = PackedInts.bitsRequired(max);
+ if (bitsPerValue == 3) {
+ bitsPerValue = 4;
+ } else if (bitsPerValue > 4) {
+ bitsPerValue = 8;
+ }
+ return new FormatAndBits(format, bitsPerValue);
+ }
+
+ private void addConstant(long constant) throws IOException {
+ meta.writeVInt(0);
+ meta.writeByte(CONST_COMPRESSED);
+ meta.writeLong(constant);
+ }
+
+ private void addUncompressed(Iterable<Number> values, int count) throws IOException {
+ meta.writeVInt(count);
+ meta.writeByte(UNCOMPRESSED); // uncompressed byte[]
+ meta.writeLong(data.getFilePointer());
+ for (Number nv : values) {
+ data.writeByte((byte) nv.longValue());
+ }
+ }
+
+ private void addTableCompressed(Iterable<Number> values, FormatAndBits compression, int count, NormMap uniqueValues) throws IOException {
+ meta.writeVInt(count);
+ meta.writeByte(TABLE_COMPRESSED); // table-compressed
+ meta.writeLong(data.getFilePointer());
+ data.writeVInt(PackedInts.VERSION_CURRENT);
+
+ long[] decode = uniqueValues.getDecodeTable();
+ // upgrade to power of two sized array
+ int size = 1 << compression.bitsPerValue;
+ data.writeVInt(size);
+ for (int i = 0; i < decode.length; i++) {
+ data.writeLong(decode[i]);
+ }
+ for (int i = decode.length; i < size; i++) {
+ data.writeLong(0);
+ }
+
+ data.writeVInt(compression.format.getId());
+ data.writeVInt(compression.bitsPerValue);
+
+ final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, compression.format, count, compression.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
+ for(Number nv : values) {
+ writer.add(uniqueValues.getOrd(nv.longValue()));
+ }
+ writer.finish();
+ }
+
+ private void addDeltaCompressed(Iterable<Number> values, int count) throws IOException {
+ meta.writeVInt(count);
+ 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();
+ }
+
+ private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, int missingCount) throws IOException {
+ meta.writeVInt(count - missingCount);
+ meta.writeByte(INDIRECT);
+ meta.writeLong(data.getFilePointer());
+ data.writeVInt(PackedInts.VERSION_CURRENT);
+ data.writeVInt(BLOCK_SIZE);
+
+ // write docs with value
+ final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+ int doc = 0;
+ for (Number n : values) {
+ long v = n.longValue();
+ if (v != 0) {
+ writer.add(doc);
+ }
+ doc++;
+ }
+ writer.finish();
+
+ // write actual values
+ addNormsField(field, new Iterable<Number>() {
+ @Override
+ public Iterator<Number> iterator() {
+ return new FilterIterator<Number,Number>(values.iterator()) {
+ @Override
+ protected boolean predicateFunction(Number value) {
+ return value.longValue() != 0;
+ }
+ };
+ }
+ });
+ }
+
+ @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;
+ }
+ }
+
+ // specialized deduplication of long->ord for norms: 99.99999% of the time this will be a single-byte range.
+ static class NormMap {
+ // we use short: at most we will add 257 values to this map before its rejected as too big above.
+ final short[] singleByteRange = new short[256];
+ final Map<Long,Short> other = new HashMap<Long,Short>();
+ int size;
+
+ {
+ Arrays.fill(singleByteRange, (short)-1);
+ }
+
+ /** adds an item to the mapping. returns true if actually added */
+ public boolean add(long l) {
+ assert size <= 256; // once we add > 256 values, we nullify the map in addNumericField and don't use this strategy
+ if (l >= Byte.MIN_VALUE && l <= Byte.MAX_VALUE) {
+ int index = (int) (l + 128);
+ short previous = singleByteRange[index];
+ if (previous < 0) {
+ singleByteRange[index] = (short) size;
+ size++;
+ return true;
+ } else {
+ return false;
+ }
+ } else {
+ if (!other.containsKey(l)) {
+ other.put(l, (short)size);
+ size++;
+ return true;
+ } else {
+ return false;
+ }
+ }
+ }
+
+ /** gets the ordinal for a previously added item */
+ public int getOrd(long l) {
+ if (l >= Byte.MIN_VALUE && l <= Byte.MAX_VALUE) {
+ int index = (int) (l + 128);
+ return singleByteRange[index];
+ } else {
+ // NPE if something is screwed up
+ return other.get(l);
+ }
+ }
+
+ /** retrieves the ordinal table for previously added items */
+ public long[] getDecodeTable() {
+ long decode[] = new long[size];
+ for (int i = 0; i < singleByteRange.length; i++) {
+ short s = singleByteRange[i];
+ if (s >= 0) {
+ decode[s] = i - 128;
+ }
+ }
+ for (Map.Entry<Long,Short> entry : other.entrySet()) {
+ decode[entry.getValue()] = entry.getKey();
+ }
+ return decode;
+ }
+ }
+}
Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java?rev=1628382&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java Tue Sep 30 09:37:52 2014
@@ -0,0 +1,125 @@
+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.NormsConsumer;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.NormsProducer;
+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 5.0 Score normalization format.
+ * <p>
+ * Encodes normalization values with these strategies:
+ * <p>
+ * <ul>
+ * <li>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.
+ * <li>Constant: when there is only one value present for the entire field, no actual data
+ * is written: this constant is encoded in the metadata
+ * <li>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}).
+ * <li>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.
+ * <li>Indirect: when norms are extremely sparse, missing values are omitted.
+ * Access to an individual value is slower, but missing norm values are never accessed
+ * by search code.
+ * </ul>
+ * <p>
+ * Files:
+ * <ol>
+ * <li><tt>.nvd</tt>: Norms data</li>
+ * <li><tt>.nvm</tt>: Norms metadata</li>
+ * </ol>
+ * <ol>
+ * <li><a name="nvm" id="nvm"></a>
+ * <p>The Norms metadata or .nvm file.</p>
+ * <p>For each norms field, this stores metadata, such as the offset into the
+ * Norms data (.nvd)</p>
+ * <p>Norms metadata (.dvm) --> Header,<Entry><sup>NumFields</sup>,Footer</p>
+ * <ul>
+ * <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
+ * <li>Entry --> FieldNumber,Type,Offset</li>
+ * <li>FieldNumber --> {@link DataOutput#writeVInt vInt}</li>
+ * <li>Type --> {@link DataOutput#writeByte Byte}</li>
+ * <li>Offset --> {@link DataOutput#writeLong Int64}</li>
+ * <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}</li>
+ * </ul>
+ * <p>FieldNumber of -1 indicates the end of metadata.</p>
+ * <p>Offset is the pointer to the start of the data in the norms data (.nvd), or the singleton value for Constant</p>
+ * <p>Type indicates how Numeric values will be compressed:
+ * <ul>
+ * <li>0 --> delta-compressed. For each block of 16k integers, every integer is delta-encoded
+ * from the minimum value within the block.
+ * <li>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.
+ * <li>2 --> constant. When there is a single value for the entire field.
+ * <li>3 --> uncompressed: Values written as a simple byte[].
+ * <li>4 --> indirect. Only documents with a value are written with a sparse encoding.
+ * </ul>
+ * <li><a name="nvd" id="nvd"></a>
+ * <p>The Norms data or .nvd file.</p>
+ * <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)</p>
+ * <p>Norms data (.nvd) --> Header,<Uncompressed | TableCompressed | DeltaCompressed><sup>NumFields</sup>,Footer</p>
+ * <ul>
+ * <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
+ * <li>Uncompressed --> {@link DataOutput#writeByte Byte}<sup>maxDoc</sup></li>
+ * <li>TableCompressed --> PackedIntsVersion,Table,BitPackedData</li>
+ * <li>Table --> TableSize, {@link DataOutput#writeLong int64}<sup>TableSize</sup></li>
+ * <li>BitpackedData --> {@link PackedInts}</li>
+ * <li>DeltaCompressed --> PackedIntsVersion,BlockSize,DeltaCompressedData</li>
+ * <li>DeltaCompressedData --> {@link BlockPackedWriter BlockPackedWriter(blockSize=16k)}</li>
+ * <li>PackedIntsVersion,BlockSize,TableSize --> {@link DataOutput#writeVInt vInt}</li>
+ * <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}</li>
+ * </ul>
+ * </ol>
+ * @lucene.experimental
+ */
+public class Lucene50NormsFormat extends NormsFormat {
+
+ /** Sole Constructor */
+ public Lucene50NormsFormat() {}
+
+ @Override
+ public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
+ return new Lucene50NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+ }
+
+ @Override
+ public NormsProducer normsProducer(SegmentReadState state) throws IOException {
+ return new Lucene50NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+ }
+
+ private static final String DATA_CODEC = "Lucene50NormsData";
+ private static final String DATA_EXTENSION = "nvd";
+ private static final String METADATA_CODEC = "Lucene50NormsMetadata";
+ private static final String METADATA_EXTENSION = "nvm";
+ static final int VERSION_START = 0;
+ static final int VERSION_CURRENT = VERSION_START;
+}
Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java?rev=1628382&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java Tue Sep 30 09:37:52 2014
@@ -0,0 +1,320 @@
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NormsProducer;
+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.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+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.MonotonicBlockPackedReader;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_START;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.CONST_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.TABLE_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.INDIRECT;
+
+/**
+ * Reader for {@link Lucene50NormsFormat}
+ */
+class Lucene50NormsProducer extends NormsProducer {
+ // metadata maps (just file pointers and minimal stuff)
+ private final Map<String,NormsEntry> norms = new HashMap<>();
+ private final IndexInput data;
+
+ // ram instances we have already loaded
+ final Map<String,NumericDocValues> instances = new HashMap<>();
+ final Map<String,Accountable> instancesInfo = new HashMap<>();
+
+ private final AtomicLong ramBytesUsed;
+ private final AtomicInteger activeCount = new AtomicInteger();
+
+ private final boolean merging;
+
+ // clone for merge: when merging we don't do any instances.put()s
+ Lucene50NormsProducer(Lucene50NormsProducer original) {
+ assert Thread.holdsLock(original);
+ norms.putAll(original.norms);
+ data = original.data.clone();
+ instances.putAll(original.instances);
+ instancesInfo.putAll(original.instancesInfo);
+ ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
+ activeCount.set(original.activeCount.get());
+ merging = true;
+ }
+
+ Lucene50NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+ merging = false;
+ 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)) {
+ Throwable priorE = null;
+ try {
+ version = CodecUtil.checkSegmentHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId());
+ readFields(in, state.fieldInfos);
+ } catch (Throwable exception) {
+ priorE = exception;
+ } finally {
+ CodecUtil.checkFooter(in, priorE);
+ }
+ }
+
+ String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+ this.data = state.directory.openInput(dataName, state.context);
+ boolean success = false;
+ try {
+ final int version2 = CodecUtil.checkSegmentHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId());
+ if (version != version2) {
+ throw new CorruptIndexException("Format versions mismatch: meta=" + version + ",data=" + version2, data);
+ }
+
+ // NOTE: data file is too costly to verify checksum against all the bytes on open,
+ // but for now we at least verify proper structure of the checksum footer: which looks
+ // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+ // such as file truncation.
+ CodecUtil.retrieveChecksum(data);
+
+ 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, meta);
+ } else if (!info.hasNorms()) {
+ throw new CorruptIndexException("Invalid field: " + info.name, meta);
+ }
+ NormsEntry entry = readEntry(info, meta);
+ norms.put(info.name, entry);
+ fieldNumber = meta.readVInt();
+ }
+ }
+
+ private NormsEntry readEntry(FieldInfo info, IndexInput meta) throws IOException {
+ NormsEntry entry = new NormsEntry();
+ entry.count = meta.readVInt();
+ entry.format = meta.readByte();
+ entry.offset = meta.readLong();
+ switch(entry.format) {
+ case CONST_COMPRESSED:
+ case UNCOMPRESSED:
+ case TABLE_COMPRESSED:
+ case DELTA_COMPRESSED:
+ break;
+ case INDIRECT:
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta);
+ }
+ entry.nested = readEntry(info, meta);
+ break;
+ default:
+ throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+ }
+ return entry;
+ }
+
+ @Override
+ public synchronized NumericDocValues getNorms(FieldInfo field) throws IOException {
+ NumericDocValues instance = instances.get(field.name);
+ if (instance == null) {
+ LoadedNorms loaded = loadNorms(norms.get(field.name));
+ instance = loaded.norms;
+ if (!merging) {
+ instances.put(field.name, instance);
+ activeCount.incrementAndGet();
+ ramBytesUsed.addAndGet(loaded.ramBytesUsed);
+ instancesInfo.put(field.name, loaded.info);
+ }
+ }
+ return instance;
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ return ramBytesUsed.get();
+ }
+
+ @Override
+ public synchronized Iterable<? extends Accountable> getChildResources() {
+ return Accountables.namedAccountables("field", instancesInfo);
+ }
+
+ @Override
+ public void checkIntegrity() throws IOException {
+ CodecUtil.checksumEntireFile(data);
+ }
+
+ private LoadedNorms loadNorms(NormsEntry entry) throws IOException {
+ LoadedNorms instance = new LoadedNorms();
+ switch(entry.format) {
+ case CONST_COMPRESSED: {
+ final long v = entry.offset;
+ instance.info = Accountables.namedAccountable("constant", 8);
+ instance.ramBytesUsed = 8;
+ instance.norms = new NumericDocValues() {
+ @Override
+ public long get(int docID) {
+ return v;
+ }
+ };
+ break;
+ }
+ case UNCOMPRESSED: {
+ data.seek(entry.offset);
+ final byte bytes[] = new byte[entry.count];
+ data.readBytes(bytes, 0, bytes.length);
+ instance.info = Accountables.namedAccountable("byte array", bytes.length);
+ instance.ramBytesUsed = RamUsageEstimator.sizeOf(bytes);
+ instance.norms = new NumericDocValues() {
+ @Override
+ public long get(int docID) {
+ return bytes[docID];
+ }
+ };
+ break;
+ }
+ case DELTA_COMPRESSED: {
+ data.seek(entry.offset);
+ int packedIntsVersion = data.readVInt();
+ int blockSize = data.readVInt();
+ final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, entry.count, false);
+ instance.info = Accountables.namedAccountable("delta compressed", reader);
+ instance.ramBytesUsed = reader.ramBytesUsed();
+ instance.norms = reader;
+ break;
+ }
+ case TABLE_COMPRESSED: {
+ data.seek(entry.offset);
+ int packedIntsVersion = data.readVInt();
+ int size = data.readVInt();
+ if (size > 256) {
+ throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + size, 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), packedIntsVersion, entry.count, bitsPerValue);
+ instance.info = Accountables.namedAccountable("table compressed", ordsReader);
+ instance.ramBytesUsed = RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed();
+ instance.norms = new NumericDocValues() {
+ @Override
+ public long get(int docID) {
+ return decode[(int)ordsReader.get(docID)];
+ }
+ };
+ break;
+ }
+ case INDIRECT: {
+ data.seek(entry.offset);
+ int packedIntsVersion = data.readVInt();
+ int blockSize = data.readVInt();
+ final MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, false);
+ LoadedNorms nestedInstance = loadNorms(entry.nested);
+ instance.ramBytesUsed = live.ramBytesUsed() + nestedInstance.ramBytesUsed;
+ instance.info = Accountables.namedAccountable("indirect -> " + nestedInstance.info, instance.ramBytesUsed);
+ final NumericDocValues values = nestedInstance.norms;
+ final int upperBound = entry.count-1;
+ instance.norms = new NumericDocValues() {
+ @Override
+ public long get(int docID) {
+ int low = 0;
+ int high = upperBound;
+ while (low <= high) {
+ int mid = (low + high) >>> 1;
+ long doc = live.get(mid);
+
+ if (doc < docID) {
+ low = mid + 1;
+ } else if (doc > docID) {
+ high = mid - 1;
+ } else {
+ return values.get(mid);
+ }
+ }
+ return 0;
+ }
+ };
+ break;
+ }
+ default:
+ throw new AssertionError();
+ }
+ return instance;
+ }
+
+ @Override
+ public void close() throws IOException {
+ data.close();
+ }
+
+ static class NormsEntry {
+ byte format;
+ long offset;
+ int count;
+ NormsEntry nested;
+ }
+
+ static class LoadedNorms {
+ NumericDocValues norms;
+ long ramBytesUsed;
+ Accountable info;
+ }
+
+ @Override
+ public synchronized NormsProducer getMergeInstance() throws IOException {
+ return new Lucene50NormsProducer(this);
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "(fields=" + norms.size() + ",active=" + activeCount.get() + ")";
+ }
+}
Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Tue Sep 30 09:37:52 2014
@@ -225,6 +225,24 @@ public abstract class PerFieldDocValuesF
private final Map<String,DocValuesProducer> fields = new TreeMap<>();
private final Map<String,DocValuesProducer> formats = new HashMap<>();
+
+ // clone for merge
+ FieldsReader(FieldsReader other) throws IOException {
+ Map<DocValuesProducer,DocValuesProducer> oldToNew = new IdentityHashMap<>();
+ // First clone all formats
+ for(Map.Entry<String,DocValuesProducer> ent : other.formats.entrySet()) {
+ DocValuesProducer values = ent.getValue().getMergeInstance();
+ formats.put(ent.getKey(), values);
+ oldToNew.put(ent.getValue(), values);
+ }
+
+ // Then rebuild fields:
+ for(Map.Entry<String,DocValuesProducer> ent : other.fields.entrySet()) {
+ DocValuesProducer producer = oldToNew.get(ent.getValue());
+ assert producer != null;
+ fields.put(ent.getKey(), producer);
+ }
+ }
public FieldsReader(final SegmentReadState readState) throws IOException {
@@ -259,24 +277,6 @@ public abstract class PerFieldDocValuesF
}
}
- private FieldsReader(FieldsReader other) {
-
- Map<DocValuesProducer,DocValuesProducer> oldToNew = new IdentityHashMap<>();
- // First clone all formats
- for(Map.Entry<String,DocValuesProducer> ent : other.formats.entrySet()) {
- DocValuesProducer values = ent.getValue();
- formats.put(ent.getKey(), values);
- oldToNew.put(ent.getValue(), values);
- }
-
- // Then rebuild fields:
- for(Map.Entry<String,DocValuesProducer> ent : other.fields.entrySet()) {
- DocValuesProducer producer = oldToNew.get(ent.getValue());
- assert producer != null;
- fields.put(ent.getKey(), producer);
- }
- }
-
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
DocValuesProducer producer = fields.get(field.name);
@@ -319,11 +319,6 @@ public abstract class PerFieldDocValuesF
}
@Override
- public DocValuesProducer clone() {
- return new FieldsReader(this);
- }
-
- @Override
public long ramBytesUsed() {
long size = 0;
for (Map.Entry<String,DocValuesProducer> entry : formats.entrySet()) {
@@ -346,6 +341,11 @@ public abstract class PerFieldDocValuesF
}
@Override
+ public DocValuesProducer getMergeInstance() throws IOException {
+ return new FieldsReader(this);
+ }
+
+ @Override
public String toString() {
return "PerFieldDocValues(formats=" + formats.size() + ")";
}
Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Tue Sep 30 09:37:52 2014
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
+import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -215,6 +216,24 @@ public abstract class PerFieldPostingsFo
private final Map<String,FieldsProducer> fields = new TreeMap<>();
private final Map<String,FieldsProducer> formats = new HashMap<>();
+
+ // clone for merge
+ FieldsReader(FieldsReader other) throws IOException {
+ Map<FieldsProducer,FieldsProducer> oldToNew = new IdentityHashMap<>();
+ // First clone all formats
+ for(Map.Entry<String,FieldsProducer> ent : other.formats.entrySet()) {
+ FieldsProducer values = ent.getValue().getMergeInstance();
+ formats.put(ent.getKey(), values);
+ oldToNew.put(ent.getValue(), values);
+ }
+
+ // Then rebuild fields:
+ for(Map.Entry<String,FieldsProducer> ent : other.fields.entrySet()) {
+ FieldsProducer producer = oldToNew.get(ent.getValue());
+ assert producer != null;
+ fields.put(ent.getKey(), producer);
+ }
+ }
public FieldsReader(final SegmentReadState readState) throws IOException {
@@ -294,6 +313,11 @@ public abstract class PerFieldPostingsFo
}
@Override
+ public FieldsProducer getMergeInstance() throws IOException {
+ return new FieldsReader(this);
+ }
+
+ @Override
public String toString() {
return "PerFieldPostings(formats=" + formats.size() + ")";
}
Added: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java?rev=1628382&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java Tue Sep 30 09:37:52 2014
@@ -0,0 +1,34 @@
+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.index.BaseNormsFormatTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests Lucene49NormsFormat
+ */
+public class TestLucene50NormsFormat extends BaseNormsFormatTestCase {
+ private final Codec codec = TestUtil.getDefaultCodec();
+
+ @Override
+ protected Codec getCodec() {
+ return codec;
+ }
+}
Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java Tue Sep 30 09:37:52 2014
@@ -301,6 +301,11 @@ public class AssertingDocValuesFormat ex
}
@Override
+ public DocValuesProducer getMergeInstance() throws IOException {
+ return new AssertingDocValuesProducer(in.getMergeInstance(), maxDoc);
+ }
+
+ @Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java Tue Sep 30 09:37:52 2014
@@ -124,6 +124,11 @@ public class AssertingNormsFormat extend
}
@Override
+ public NormsProducer getMergeInstance() throws IOException {
+ return new AssertingNormsProducer(in.getMergeInstance(), maxDoc);
+ }
+
+ @Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java Tue Sep 30 09:37:52 2014
@@ -111,6 +111,11 @@ public final class AssertingPostingsForm
}
@Override
+ public FieldsProducer getMergeInstance() throws IOException {
+ return new AssertingFieldsProducer(in.getMergeInstance());
+ }
+
+ @Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java Tue Sep 30 09:37:52 2014
@@ -97,6 +97,11 @@ public class AssertingStoredFieldsFormat
}
@Override
+ public StoredFieldsReader getMergeInstance() throws IOException {
+ return new AssertingStoredFieldsReader(in.getMergeInstance(), maxDoc);
+ }
+
+ @Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java Tue Sep 30 09:37:52 2014
@@ -96,6 +96,11 @@ public class AssertingTermVectorsFormat
}
@Override
+ public TermVectorsReader getMergeInstance() throws IOException {
+ return new AssertingTermVectorsReader(in.getMergeInstance());
+ }
+
+ @Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java?rev=1628382&r1=1628381&r2=1628382&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java Tue Sep 30 09:37:52 2014
@@ -139,6 +139,19 @@ public abstract class BaseNormsFormatTes
}
}
+ public void testSparse() 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.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : 0;
+ }
+ });
+ }
+ }
+
private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception {
int numDocs = atLeast(500);
long norms[] = new long[numDocs];