LUCENE-7475: Make norms sparse.

This commit is contained in:
Adrien Grand 2016-10-05 14:07:15 +02:00
parent 28d187acd1
commit 9128bdbaf5
36 changed files with 1219 additions and 107 deletions

View File

@ -22,6 +22,9 @@ API Changes
to iterators, enabling future codec compression improvements. (Mike
McCandless)
* LUCENE-7475: Norms now support sparsity, allowing to pay for what is
actually used. (Adrien Grand)
Bug Fixes
Improvements

View File

@ -74,7 +74,7 @@ public class Lucene53NormsFormat extends NormsFormat {
@Override
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
return new Lucene53NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
throw new UnsupportedOperationException("This format can only be used for reading");
}
@Override
@ -82,10 +82,10 @@ public class Lucene53NormsFormat extends NormsFormat {
return new Lucene53NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
private static final String DATA_CODEC = "Lucene53NormsData";
private static final String DATA_EXTENSION = "nvd";
private static final String METADATA_CODEC = "Lucene53NormsMetadata";
private static final String METADATA_EXTENSION = "nvm";
static final String DATA_CODEC = "Lucene53NormsData";
static final String DATA_EXTENSION = "nvd";
static final String METADATA_CODEC = "Lucene53NormsMetadata";
static final String METADATA_EXTENSION = "nvm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
}

View File

@ -17,7 +17,7 @@
/**
* Components from the Lucene 5.3 index format
* See {@link org.apache.lucene.codecs.lucene54} for an overview
* See {@link org.apache.lucene.codecs.lucene53} for an overview
* of the index format.
*/
package org.apache.lucene.codecs.lucene53;

View File

@ -171,7 +171,7 @@ public class Lucene60Codec extends Codec {
private final NormsFormat normsFormat = new Lucene53NormsFormat();
@Override
public final NormsFormat normsFormat() {
public NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -32,9 +32,9 @@ import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
@ -170,7 +170,7 @@ public class Lucene62Codec extends Codec {
private final NormsFormat normsFormat = new Lucene53NormsFormat();
@Override
public final NormsFormat normsFormat() {
public NormsFormat normsFormat() {
return normsFormat;
}
}

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 6.2 file format.
</body>
</html>

View File

@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene60.Lucene60Codec
org.apache.lucene.codecs.lucene62.Lucene62Codec

View File

@ -13,3 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat

View File

@ -58,7 +58,7 @@ class Lucene53NormsConsumer extends NormsConsumer {
@Override
public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
addNormsField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, maxDoc));
addNormsField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, maxDoc, true));
}
private void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {

View File

@ -0,0 +1,31 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene53;
import java.io.IOException;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.index.SegmentWriteState;
public class Lucene53RWNormsFormat extends Lucene53NormsFormat {
@Override
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
return new Lucene53NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
}

View File

@ -16,19 +16,23 @@
*/
package org.apache.lucene.codecs.lucene53;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.lucene.codecs.lucene62.Lucene62RWCodec;
import org.apache.lucene.index.BaseNormsFormatTestCase;
/**
* Tests Lucene53NormsFormat
*/
public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
private final Codec codec = new Lucene70Codec();
private final Codec codec = new Lucene62RWCodec();
@Override
protected Codec getCodec() {
return codec;
}
@Override
protected boolean codecSupportsSparsity() {
return false;
}
}

View File

@ -0,0 +1,32 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene62;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.lucene53.Lucene53RWNormsFormat;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
public class Lucene62RWCodec extends Lucene62Codec {
private final NormsFormat normsFormat = new Lucene53RWNormsFormat();
@Override
public NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -187,7 +187,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
};
}
private Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
public Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
final OneField field = fields.get(fieldInfo.name);
final IndexInput in = data.clone();
final BytesRefBuilder scratch = new BytesRefBuilder();

View File

@ -30,7 +30,6 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
/**
* plain-text norms format.
@ -70,7 +69,7 @@ public class SimpleTextNormsFormat extends NormsFormat {
@Override
public NumericDocValues getNorms(FieldInfo field) throws IOException {
return new LegacyNumericDocValuesWrapper(new Bits.MatchAllBits(impl.maxDoc), impl.getNumericNonIterator(field));
return new LegacyNumericDocValuesWrapper(impl.getNumericDocsWithField(field), impl.getNumericNonIterator(field));
}
@Override
@ -117,7 +116,7 @@ public class SimpleTextNormsFormat extends NormsFormat {
@Override
public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
impl.addNumericField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, impl.numDocs));
impl.addNumericField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, impl.numDocs, false));
}
@Override

View File

@ -372,7 +372,8 @@ public class LegacyDocValuesIterables {
*
* @deprecated Consume {@link NumericDocValues} instead. */
@Deprecated
public static Iterable<Number> normsIterable(final FieldInfo field, final NormsProducer normsProducer, final int maxDoc) {
public static Iterable<Number> normsIterable(final FieldInfo field,
final NormsProducer normsProducer, final int maxDoc, boolean missingAsZero) {
return new Iterable<Number>() {
@ -411,9 +412,11 @@ public class LegacyDocValuesIterables {
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
} else {
} else if (missingAsZero) {
// Unlike NumericDocValues, norms should return for missing values:
result = 0;
} else {
result = null;
}
return result;
}

View File

@ -17,7 +17,7 @@
/**
* Components from the Lucene 5.0 index format
* See {@link org.apache.lucene.codecs.lucene53} for an overview
* See {@link org.apache.lucene.codecs.lucene50} for an overview
* of the index format.
*/
package org.apache.lucene.codecs.lucene50;

View File

@ -35,7 +35,6 @@ import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
import org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat;
@ -168,7 +167,7 @@ public class Lucene70Codec extends Codec {
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70");
private final NormsFormat normsFormat = new Lucene53NormsFormat();
private final NormsFormat normsFormat = new Lucene70NormsFormat();
@Override
public final NormsFormat normsFormat() {

View File

@ -0,0 +1,155 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import static org.apache.lucene.codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
/**
* Writer for {@link Lucene70NormsFormat}
*/
final class Lucene70NormsConsumer extends NormsConsumer {
IndexOutput data, meta;
final int maxDoc;
Lucene70NormsConsumer(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.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
meta = state.directory.createOutput(metaName, state.context);
CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
maxDoc = state.segmentInfo.maxDoc();
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this);
}
}
}
@Override
public void close() throws IOException {
boolean success = false;
try {
if (meta != null) {
meta.writeInt(-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 addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
NumericDocValues values = normsProducer.getNorms(field);
int numDocsWithValue = 0;
long min = Long.MAX_VALUE;
long max = Long.MIN_VALUE;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
numDocsWithValue++;
long v = values.longValue();
min = Math.min(min, v);
max = Math.max(max, v);
}
assert numDocsWithValue <= maxDoc;
meta.writeInt(field.number);
if (numDocsWithValue == 0) {
meta.writeLong(-2);
} else if (numDocsWithValue == maxDoc) {
meta.writeLong(-1);
} else {
meta.writeLong(data.getFilePointer());
values = normsProducer.getNorms(field);
SparseDISI.writeBitSet(values, maxDoc, data);
}
meta.writeInt(numDocsWithValue);
int numBytesPerValue = numBytesPerValue(min, max);
meta.writeByte((byte) numBytesPerValue);
if (numBytesPerValue == 0) {
meta.writeLong(min);
} else {
meta.writeLong(data.getFilePointer());
values = normsProducer.getNorms(field);
writeValues(values, numBytesPerValue, data);
}
}
private int numBytesPerValue(long min, long max) {
if (min >= max) {
return 0;
} else if (min >= Byte.MIN_VALUE && max <= Byte.MAX_VALUE) {
return 1;
} else if (min >= Short.MIN_VALUE && max <= Short.MAX_VALUE) {
return 2;
} else if (min >= Integer.MIN_VALUE && max <= Integer.MAX_VALUE) {
return 4;
} else {
return 8;
}
}
private void writeValues(NumericDocValues values, int numBytesPerValue, IndexOutput out) throws IOException, AssertionError {
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
long value = values.longValue();
switch (numBytesPerValue) {
case 1:
out.writeByte((byte) value);
break;
case 2:
out.writeShort((short) value);
break;
case 4:
out.writeInt((int) value);
break;
case 8:
out.writeLong(value);
break;
default:
throw new AssertionError();
}
}
}
}

View File

@ -0,0 +1,97 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
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;
/**
* Lucene 7.0 Score normalization format.
* <p>
* Encodes normalization values by encoding each value with the minimum
* number of bytes needed to represent the range (which can be zero).
* <p>
* Files:
* <ol>
* <li><tt>.nvd</tt>: Norms data</li>
* <li><tt>.nvm</tt>: Norms metadata</li>
* </ol>
* <ol>
* <li><a name="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) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
* <li>Entry --&gt; FieldNumber, DocsWithFieldAddress, NumDocsWithField, BytesPerNorm, NormsAddress</li>
* <li>FieldNumber --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>DocsWithFieldAddress --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>NumDocsWithField --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>BytesPerNorm --&gt; {@link DataOutput#writeByte byte}</li>
* <li>NormsAddress --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* <p>FieldNumber of -1 indicates the end of metadata.</p>
* <p>NormsAddress is the pointer to the start of the data in the norms data (.nvd), or the singleton value
* when BytesPerValue = 0. If BytesPerValue is different from 0 then there are NumDocsWithField values
* to read at that offset.</p>
* <p>DocsWithFieldAddress is the pointer to the start of the bit set containing documents that have a norm
* in the norms data (.nvd), or -2 if no documents have a norm value, or -1 if all documents have a norm
* value.</p>
* <li><a name="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) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer</p>
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
* <li>DocsWithFieldData --&gt; Bit set of MaxDoc bits</li>
* <li>NormsData --&gt; {@link DataOutput#writeByte(byte) byte}<sup>NumDocsWithField * BytesPerValue</sup></li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* </ol>
* @lucene.experimental
*/
public class Lucene70NormsFormat extends NormsFormat {
/** Sole Constructor */
public Lucene70NormsFormat() {}
@Override
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
return new Lucene70NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
@Override
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
return new Lucene70NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
private static final String DATA_CODEC = "Lucene70NormsData";
private static final String DATA_EXTENSION = "nvd";
private static final String METADATA_CODEC = "Lucene70NormsMetadata";
private static final String METADATA_EXTENSION = "nvm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
}

View File

@ -0,0 +1,271 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import static org.apache.lucene.codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
import static org.apache.lucene.codecs.lucene70.Lucene70NormsFormat.VERSION_START;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues;
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.store.RandomAccessInput;
import org.apache.lucene.util.IOUtils;
/**
* Reader for {@link Lucene70NormsFormat}
*/
final class Lucene70NormsProducer extends NormsProducer {
// metadata maps (just file pointers and minimal stuff)
private final Map<Integer,NormsEntry> norms = new HashMap<>();
private final IndexInput data;
private final int maxDoc;
Lucene70NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
maxDoc = state.segmentInfo.maxDoc();
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
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.checkIndexHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
readFields(in, state.fieldInfos);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(in, priorE);
}
}
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = state.directory.openInput(dataName, state.context);
boolean success = false;
try {
final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
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);
}
}
}
static class NormsEntry {
byte bytesPerNorm;
long docsWithFieldOffset;
int numDocsWithField;
long normsOffset;
}
static abstract class LongValues {
abstract long get(int index) throws IOException;
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
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 = new NormsEntry();
entry.docsWithFieldOffset = meta.readLong();
entry.numDocsWithField = meta.readInt();
entry.bytesPerNorm = meta.readByte();
switch (entry.bytesPerNorm) {
case 0: case 1: case 2: case 4: case 8:
break;
default:
throw new CorruptIndexException("Invalid bytesPerValue: " + entry.bytesPerNorm + ", field: " + info.name, meta);
}
entry.normsOffset = meta.readLong();
norms.put(info.number, entry);
}
}
@Override
public NumericDocValues getNorms(FieldInfo field) throws IOException {
final NormsEntry entry = norms.get(field.number);
if (entry.docsWithFieldOffset == -2) {
// empty
return DocValues.emptyNumeric();
} else if (entry.docsWithFieldOffset == -1) {
// dense
final LongValues normValues = getNormValues(entry);
return new NumericDocValues() {
int doc = -1;
@Override
public long longValue() throws IOException {
return normValues.get(doc);
}
@Override
public int docID() {
return doc;
}
@Override
public int nextDoc() throws IOException {
return advance(doc + 1);
}
@Override
public int advance(int target) throws IOException {
if (target >= maxDoc) {
return doc = NO_MORE_DOCS;
}
return doc = target;
}
@Override
public long cost() {
return maxDoc;
}
};
} else {
// sparse
final LongValues normValues = getNormValues(entry);
final SparseDISI disi;
synchronized (data) {
disi = new SparseDISI(maxDoc, data, entry.docsWithFieldOffset, entry.numDocsWithField);
}
return new NumericDocValues() {
@Override
public int advance(int target) throws IOException {
return disi.advance(target);
}
@Override
public int nextDoc() throws IOException {
return disi.nextDoc();
}
@Override
public int docID() {
return disi.docID();
}
@Override
public long cost() {
return entry.numDocsWithField;
}
@Override
public long longValue() throws IOException {
return normValues.get(disi.index());
}
};
}
}
private LongValues getNormValues(NormsEntry entry) throws IOException {
if (entry.bytesPerNorm == 0) {
return new LongValues() {
@Override
long get(int index) {
return entry.normsOffset;
}
};
} else {
RandomAccessInput slice;
synchronized (data) {
slice = data.randomAccessSlice(entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm);
}
switch (entry.bytesPerNorm) {
case 1:
return new LongValues() {
@Override
long get(int index) throws IOException {
return slice.readByte(index);
}
};
case 2:
return new LongValues() {
@Override
long get(int index) throws IOException {
return slice.readShort(((long) index) << 1);
}
};
case 4:
return new LongValues() {
@Override
long get(int index) throws IOException {
return slice.readInt(((long) index) << 2);
}
};
case 8:
return new LongValues() {
@Override
long get(int index) throws IOException {
return slice.readLong(((long) index) << 3);
}
};
default:
// should not happen, we already validate bytesPerNorm in readFields
throw new AssertionError();
}
}
}
@Override
public void close() throws IOException {
data.close();
}
@Override
public long ramBytesUsed() {
return 64L * norms.size(); // good enough
}
@Override
public void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(data);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + norms.size() + ")";
}
}

View File

@ -0,0 +1,115 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import java.io.IOException;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RandomAccessInput;
final class SparseDISI extends DocIdSetIterator {
static void writeBitSet(DocIdSetIterator it, int maxDoc, IndexOutput out) throws IOException {
int currentIndex = 0;
long currentBits = 0;
for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
final int index = doc >>> 6;
if (index > currentIndex) {
out.writeLong(currentBits);
for (int i = currentIndex + 1; i < index; ++i) {
out.writeLong(0L);
}
currentIndex = index;
currentBits = 0L;
}
currentBits |= 1L << doc;
}
out.writeLong(currentBits);
final int maxIndex = (maxDoc - 1) >>> 6;
for (int i = currentIndex + 1; i <= maxIndex; ++i) {
out.writeLong(0L);
}
}
final int maxDoc;
final int numWords;
final long cost;
final RandomAccessInput slice;
int doc = -1;
int wordIndex = -1;
long word;
int index = -1;
SparseDISI(int maxDoc, IndexInput in, long offset, long cost) throws IOException {
this.maxDoc = maxDoc;
this.numWords = (int) ((maxDoc + 63L) >>> 6);
this.slice = in.randomAccessSlice(offset, numWords * 8L);
this.cost = cost;
}
@Override
public int advance(int target) throws IOException {
if (target >= maxDoc) {
return doc = NO_MORE_DOCS;
}
final int targetWordIndex = target >>> 6;
for (int i = wordIndex + 1; i <= targetWordIndex; ++i) {
word = slice.readLong(i << 3);
index += Long.bitCount(word);
}
wordIndex = targetWordIndex;
long leftBits = word >>> target;
if (leftBits != 0L) {
return doc = target + Long.numberOfTrailingZeros(leftBits);
}
while (++wordIndex < numWords) {
word = slice.readLong(wordIndex << 3);
if (word != 0) {
index += Long.bitCount(word);
return doc = (wordIndex << 6) + Long.numberOfTrailingZeros(word);
}
}
return doc = NO_MORE_DOCS;
}
@Override
public int nextDoc() throws IOException {
return advance(doc + 1);
}
@Override
public int docID() {
return doc;
}
@Override
public long cost() {
return cost;
}
public int index() {
return index - Long.bitCount(word >>> doc) + 1;
}
}

View File

@ -163,7 +163,7 @@
* all documents omit position data.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}.
* {@link org.apache.lucene.codecs.lucene70.Lucene70NormsFormat Normalization factors}.
* For each field in each document, a value is stored
* that is multiplied into the score for hits on that field.
* </li>
@ -278,12 +278,12 @@
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
* <td>{@link org.apache.lucene.codecs.lucene70.Lucene70NormsFormat Norms}</td>
* <td>.nvd, .nvm</td>
* <td>Encodes length and boost factors for docs and fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}</td>
* <td>{@link org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat Per-Document Values}</td>
* <td>.dvd, .dvm</td>
* <td>Encodes additional scoring factors or other per-document information.</td>
* </tr>

View File

@ -665,8 +665,17 @@ final class DefaultIndexingChain extends DocConsumer {
}
public void finish() throws IOException {
if (fieldInfo.omitsNorms() == false && invertState.length != 0) {
norms.addValue(docState.docID, similarity.computeNorm(invertState));
if (fieldInfo.omitsNorms() == false) {
long normValue;
if (invertState.length == 0) {
// the field exists in this document, but it did not have
// any indexed tokens, so we assign a default value of zero
// to the norm
normValue = 0;
} else {
normValue = similarity.computeNorm(invertState);
}
norms.addValue(docState.docID, normValue);
}
termsHashPerField.finish();

View File

@ -21,7 +21,10 @@ import java.io.IOException;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.BitSetIterator;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
@ -29,29 +32,34 @@ import org.apache.lucene.util.packed.PackedLongValues;
* segment flushes. */
class NormValuesWriter {
private final static long MISSING = 0L;
private FixedBitSet docsWithField;
private PackedLongValues.Builder pending;
private final Counter iwBytesUsed;
private long bytesUsed;
private final FieldInfo fieldInfo;
private int lastDocID = -1;
public NormValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
docsWithField = new FixedBitSet(64);
pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
bytesUsed = pending.ramBytesUsed();
bytesUsed = pending.ramBytesUsed() + docsWithField.ramBytesUsed();
this.fieldInfo = fieldInfo;
this.iwBytesUsed = iwBytesUsed;
iwBytesUsed.addAndGet(bytesUsed);
}
public void addValue(int docID, long value) {
// Fill in any holes:
for (int i = (int)pending.size(); i < docID; ++i) {
pending.add(MISSING);
if (docID <= lastDocID) {
throw new IllegalArgumentException("Norm for \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
}
pending.add(value);
docsWithField = FixedBitSet.ensureCapacity(docsWithField, docID);
docsWithField.set(docID);
updateBytesUsed();
lastDocID = docID;
}
private void updateBytesUsed() {
@ -65,7 +73,6 @@ class NormValuesWriter {
public void flush(SegmentWriteState state, NormsConsumer normsConsumer) throws IOException {
final int maxDoc = state.segmentInfo.maxDoc();
final PackedLongValues values = pending.build();
normsConsumer.addNormsField(fieldInfo,
@ -75,7 +82,7 @@ class NormValuesWriter {
if (fieldInfo != NormValuesWriter.this.fieldInfo) {
throw new IllegalArgumentException("wrong fieldInfo");
}
return new BufferedNorms(maxDoc, values);
return new BufferedNorms(values, docsWithField);
}
@Override
@ -98,32 +105,24 @@ class NormValuesWriter {
// iterates over the values we have in ram
private static class BufferedNorms extends NumericDocValues {
final PackedLongValues.Iterator iter;
final int size;
final int maxDoc;
private int docID = -1;
final DocIdSetIterator docsWithField;
private long value;
BufferedNorms(int maxDoc, PackedLongValues values) {
this.maxDoc = maxDoc;
BufferedNorms(PackedLongValues values, FixedBitSet docsWithFields) {
this.iter = values.iterator();
this.size = (int) values.size();
this.docsWithField = new BitSetIterator(docsWithFields, values.size());
}
@Override
public int docID() {
return docID;
return docsWithField.docID();
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
docID = NO_MORE_DOCS;
}
if (docID < size) {
public int nextDoc() throws IOException {
int docID = docsWithField.nextDoc();
if (docID != NO_MORE_DOCS) {
value = iter.next();
} else {
value = MISSING;
}
return docID;
}
@ -135,7 +134,7 @@ class NormValuesWriter {
@Override
public long cost() {
return maxDoc;
return docsWithField.cost();
}
@Override

View File

@ -13,5 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.lucene62.Lucene62Codec
org.apache.lucene.codecs.lucene70.Lucene70Codec

View File

@ -13,5 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat
org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat

View File

@ -0,0 +1,34 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.lucene.index.BaseNormsFormatTestCase;
/**
* Tests Lucene70NormsFormat
*/
public class TestLucene70NormsFormat extends BaseNormsFormatTestCase {
private final Codec codec = new Lucene70Codec();
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -0,0 +1,94 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BitSetIterator;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
public class TestSparseDISI extends LuceneTestCase {
public void testRandom() throws IOException {
try (Directory dir = newDirectory()) {
for (int i = 0; i < 1000; ++i) {
doTestRandom(dir);
}
}
}
private void doTestRandom(Directory dir) throws IOException {
List<Integer> docs = new ArrayList<>();
final int maxStep = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 2, 10));
final int numDocs = TestUtil.nextInt(random(), 1, 1000);
for (int doc = -1, i = 0; i < numDocs; ++i) {
doc += TestUtil.nextInt(random(), 1, maxStep);
docs.add(doc);
}
final int maxDoc = docs.get(docs.size() - 1) + TestUtil.nextInt(random(), 1, 100);
FixedBitSet set = new FixedBitSet(maxDoc);
for (int doc : docs) {
set.set(doc);
}
try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) {
SparseDISI.writeBitSet(new BitSetIterator(set, docs.size()), maxDoc, out);
}
try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
SparseDISI disi = new SparseDISI(maxDoc, in, 0L, docs.size());
BitSetIterator disi2 = new BitSetIterator(set, docs.size());
int i = 0;
for (int doc = disi2.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = disi2.nextDoc()) {
assertEquals(doc, disi.nextDoc());
assertEquals(i++, disi.index());
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, disi.nextDoc());
}
for (int step : new int[] {1, 20, maxStep, maxStep * 10}) {
try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
SparseDISI disi = new SparseDISI(maxDoc, in, 0L, docs.size());
BitSetIterator disi2 = new BitSetIterator(set, docs.size());
while (true) {
int target = disi2.docID() + step;
int doc = disi2.advance(target);
assertEquals(doc, disi.advance(target));
if (doc == DocIdSetIterator.NO_MORE_DOCS) {
break;
}
int index = Collections.binarySearch(docs, doc);
assertEquals(index, disi.index());
}
}
}
dir.deleteFile("foo");
}
}

View File

@ -23,6 +23,7 @@ import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.TextField;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.TermStatistics;
@ -186,4 +187,24 @@ public class TestNorms extends LuceneTestCase {
throw new UnsupportedOperationException();
}
}
public void testEmptyValueVsNoValue() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig cfg = newIndexWriterConfig().setMergePolicy(newLogMergePolicy());
IndexWriter w = new IndexWriter(dir, cfg);
Document doc = new Document();
w.addDocument(doc);
doc.add(newTextField("foo", "", Store.NO));
w.addDocument(doc);
w.forceMerge(1);
IndexReader reader = DirectoryReader.open(w);
w.close();
LeafReader leafReader = getOnlyLeafReader(reader);
NumericDocValues normValues = leafReader.getNormValues("foo");
assertNotNull(normValues);
assertEquals(1, normValues.nextDoc()); // doc 0 does not have norms
assertEquals(0, normValues.longValue());
reader.close();
dir.close();
}
}

View File

@ -30,9 +30,11 @@ import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.TermStatistics;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.TestUtil;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
@ -47,11 +49,30 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
* test fails to catch then this test needs to be improved! */
public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCase {
/** Whether the codec supports sparse values. */
protected boolean codecSupportsSparsity() {
return true;
}
public void testByteRange() throws Exception {
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
});
}
}
public void testSparseByteRange() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
@ -64,7 +85,21 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return TestUtil.nextLong(r, Short.MIN_VALUE, Short.MAX_VALUE);
}
});
}
}
public void testSparseShortRange() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return TestUtil.nextLong(r, Short.MIN_VALUE, Short.MAX_VALUE);
@ -77,7 +112,21 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return TestUtil.nextLong(r, Long.MIN_VALUE, Long.MAX_VALUE);
}
});
}
}
public void testSparseLongRange() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return TestUtil.nextLong(r, Long.MIN_VALUE, Long.MAX_VALUE);
@ -90,7 +139,26 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
int thingToDo = r.nextInt(3);
switch (thingToDo) {
case 0: return Long.MIN_VALUE;
case 1: return Long.MAX_VALUE;
default: return TestUtil.nextLong(r, Long.MIN_VALUE, Long.MAX_VALUE);
}
}
});
}
}
public void testSparseFullLongRange() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
int thingToDo = r.nextInt(3);
@ -108,7 +176,21 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return r.nextBoolean() ? 20 : 3;
}
});
}
}
public void testFewSparseValues() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return r.nextBoolean() ? 20 : 3;
@ -121,7 +203,21 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return r.nextBoolean() ? 1000000L : -5000;
}
});
}
}
public void testFewSparseLargeValues() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return r.nextBoolean() ? 1000000L : -5000;
@ -133,7 +229,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
public void testAllZeros() throws Exception {
int iterations = atLeast(1);
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return 0;
@ -142,11 +238,24 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
}
}
public void testSparse() throws Exception {
public void testSparseAllZeros() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return 0;
}
});
}
}
public void testMostZeros() throws Exception {
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : 0;
@ -160,7 +269,22 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
final Random r = random();
for (int i = 0; i < iterations; i++) {
final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : commonValue;
}
});
}
}
public void testSparseOutliers() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : commonValue;
@ -175,7 +299,23 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
for (int i = 0; i < iterations; i++) {
final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
final long uncommonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? uncommonValue : commonValue;
}
});
}
}
public void testSparseOutliers2() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; i++) {
final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
final long uncommonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? uncommonValue : commonValue;
@ -196,7 +336,28 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
for (int j = 0; j < numOtherValues; ++j) {
otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
}
});
}
public void testSparseNCommon() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
final Random r = random();
final int N = TestUtil.nextInt(r, 2, 15);
final long[] commonValues = new long[N];
for (int j = 0; j < N; ++j) {
commonValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
final int numOtherValues = TestUtil.nextInt(r, 2, 256 - N);
final long[] otherValues = new long[numOtherValues];
for (int j = 0; j < numOtherValues; ++j) {
otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
@ -224,7 +385,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
for (int j = 0; j < numOtherValues; ++j) {
otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
doTestNormsVersusDocValues(new LongProducer() {
doTestNormsVersusDocValues(1, new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
@ -234,16 +395,61 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
}
}
private void doTestNormsVersusDocValues(LongProducer longs) throws Exception {
/**
* a more thorough n-common that tests all low bpv and sparse docs
*/
@Nightly
public void testSparseNCommonBig() throws Exception {
assumeTrue("Requires sparse norms support", codecSupportsSparsity());
final int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; ++i) {
// 16 is 4 bpv, the max before we jump to 8bpv
for (int n = 2; n < 16; ++n) {
final int N = n;
final long[] commonValues = new long[N];
for (int j = 0; j < N; ++j) {
commonValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
final int numOtherValues = TestUtil.nextInt(r, 2, 256 - N);
final long[] otherValues = new long[numOtherValues];
for (int j = 0; j < numOtherValues; ++j) {
otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
}
});
}
}
}
private void doTestNormsVersusDocValues(double density, LongProducer longs) throws Exception {
int numDocs = atLeast(500);
long norms[] = new long[numDocs];
for (int i = 0; i < numDocs; i++) {
final FixedBitSet docsWithField = new FixedBitSet(numDocs);
final int numDocsWithField = Math.max(1, (int) (density * numDocs));
if (numDocsWithField == numDocs) {
docsWithField.set(0, numDocs);
} else {
int i = 0;
while (i < numDocsWithField) {
int doc = random().nextInt(numDocs);
if (docsWithField.get(doc) == false) {
docsWithField.set(doc);
++i;
}
}
}
long norms[] = new long[numDocsWithField];
for (int i = 0; i < numDocsWithField; i++) {
norms[i] = longs.next();
}
Directory dir = newDirectory();
Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
IndexWriterConfig conf = newIndexWriterConfig(analyzer);
IndexWriterConfig conf = newIndexWriterConfig(analyzer);conf.setMergePolicy(NoMergePolicy.INSTANCE);
conf.setSimilarity(new CannedNormSimilarity(norms));
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
Document doc = new Document();
@ -254,12 +460,18 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
doc.add(indexedField);
doc.add(dvField);
for (int i = 0; i < numDocs; i++) {
for (int i = 0, j = 0; i < numDocs; i++) {
idField.setStringValue(Integer.toString(i));
long value = norms[i];
dvField.setLongValue(value);
indexedField.setStringValue(Long.toString(value));
writer.addDocument(doc);
if (docsWithField.get(i) == false) {
Document doc2 = new Document();
doc2.add(idField);
writer.addDocument(doc2);
} else {
long value = norms[j++];
dvField.setLongValue(value);
indexedField.setStringValue(Long.toString(value));
writer.addDocument(doc);
}
if (random().nextInt(31) == 0) {
writer.commit();
}
@ -280,12 +492,14 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
LeafReader r = context.reader();
NumericDocValues expected = r.getNumericDocValues("dv");
NumericDocValues actual = r.getNormValues("indexed");
for (int i = 0; i < r.maxDoc(); i++) {
assertEquals(i, expected.nextDoc());
assertEquals(i, actual.nextDoc());
assertEquals("doc " + i, expected.longValue(), actual.longValue());
assertEquals(expected == null, actual == null);
if (expected != null) {
for (int d = expected.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = expected.nextDoc()) {
assertEquals(d, actual.nextDoc());
assertEquals("doc " + d, expected.longValue(), actual.longValue());
}
assertEquals(NO_MORE_DOCS, actual.nextDoc());
}
assertEquals(NO_MORE_DOCS, expected.nextDoc());
}
ir.close();
@ -297,10 +511,13 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
LeafReader r = context.reader();
NumericDocValues expected = r.getNumericDocValues("dv");
NumericDocValues actual = r.getNormValues("indexed");
for (int i = 0; i < r.maxDoc(); i++) {
assertEquals(i, expected.nextDoc());
assertEquals(i, actual.nextDoc());
assertEquals("doc " + i, expected.longValue(), actual.longValue());
assertEquals(expected == null, actual == null);
if (expected != null) {
for (int d = expected.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = expected.nextDoc()) {
assertEquals(d, actual.nextDoc());
assertEquals("doc " + d, expected.longValue(), actual.longValue());
}
assertEquals(NO_MORE_DOCS, actual.nextDoc());
}
}
@ -403,9 +620,13 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
// be undead:
NumericDocValues norms = MultiDocValues.getNormValues(r, "content");
assertNotNull(norms);
for(int i=0;i<r.maxDoc();i++) {
assertEquals(i, norms.nextDoc());
assertEquals(0, norms.longValue());
if (codecSupportsSparsity()) {
assertEquals(DocIdSetIterator.NO_MORE_DOCS, norms.nextDoc());
} else {
for(int i=0;i<r.maxDoc();i++) {
assertEquals(i, norms.nextDoc());
assertEquals(0, norms.longValue());
}
}
r.close();