LUCENE-5969: add 5.10 dv with segment header, CONST optimization, and missingBits ghostbuster

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1632200 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2014-10-16 01:20:58 +00:00
parent 45d882f470
commit 373d007474
20 changed files with 2308 additions and 95 deletions

View File

@ -74,12 +74,12 @@ public class Lucene410Codec extends Codec {
}
@Override
public final StoredFieldsFormat storedFieldsFormat() {
public StoredFieldsFormat storedFieldsFormat() {
return fieldsFormat;
}
@Override
public final TermVectorsFormat termVectorsFormat() {
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@ -94,7 +94,7 @@ public class Lucene410Codec extends Codec {
}
@Override
public final SegmentInfoFormat segmentInfoFormat() {
public SegmentInfoFormat segmentInfoFormat() {
return segmentInfosFormat;
}
@ -127,7 +127,7 @@ public class Lucene410Codec extends Codec {
}
@Override
public final DocValuesFormat docValuesFormat() {
public DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@ -142,7 +142,7 @@ public class Lucene410Codec extends Codec {
};
@Override
public final NormsFormat normsFormat() {
public NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -28,6 +28,7 @@ import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
@ -41,7 +42,11 @@ import org.apache.lucene.util.packed.DirectWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
/** writer for {@link Lucene410DocValuesFormat} */
/**
* writer for 4.10 docvalues format
* @deprecated only for old 4.x segments
*/
@Deprecated
class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable {
static final int BLOCK_SIZE = 16384;
@ -108,6 +113,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
checkCanWrite(field);
addNumericField(field, values, true);
}
@ -262,6 +268,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
checkCanWrite(field);
// write the byte[] data
meta.writeVInt(field.number);
meta.writeByte(Lucene410DocValuesFormat.BINARY);
@ -466,6 +473,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
checkCanWrite(field);
meta.writeVInt(field.number);
meta.writeByte(Lucene410DocValuesFormat.SORTED);
addTermsDict(field, values);
@ -474,6 +482,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
checkCanWrite(field);
meta.writeVInt(field.number);
meta.writeByte(Lucene410DocValuesFormat.SORTED_NUMERIC);
if (isSingleValued(docToValueCount)) {
@ -491,6 +500,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
checkCanWrite(field);
meta.writeVInt(field.number);
meta.writeByte(Lucene410DocValuesFormat.SORTED_SET);
@ -556,4 +566,14 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
meta = data = null;
}
}
void checkCanWrite(FieldInfo field) {
if ((field.getDocValuesType() == DocValuesType.NUMERIC ||
field.getDocValuesType() == DocValuesType.BINARY) &&
field.getDocValuesGen() != -1) {
// ok
} else {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
}
}

View File

@ -0,0 +1,61 @@
package org.apache.lucene.codecs.lucene410;
/*
* 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.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* 4.10 docvalues format
* @deprecated only for old 4.x segments
*/
@Deprecated
public class Lucene410DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */
public Lucene410DocValuesFormat() {
super("Lucene410");
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene410DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
public final DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene410DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
static final String DATA_CODEC = "Lucene410DocValuesData";
static final String DATA_EXTENSION = "dvd";
static final String META_CODEC = "Lucene410ValuesMetadata";
static final String META_EXTENSION = "dvm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static final byte NUMERIC = 0;
static final byte BINARY = 1;
static final byte SORTED = 2;
static final byte SORTED_SET = 3;
static final byte SORTED_NUMERIC = 4;
}

View File

@ -74,7 +74,11 @@ import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.DirectReader;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
/** reader for {@link Lucene410DocValuesFormat} */
/**
* reader for 4.10 docvalues format
* @deprecated only for old 4.x segments
*/
@Deprecated
class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable {
private final Map<String,NumericEntry> numerics = new HashMap<>();
private final Map<String,BinaryEntry> binaries = new HashMap<>();

View File

@ -16,3 +16,5 @@
org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat
org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat
org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat
org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat

View File

@ -0,0 +1,71 @@
package org.apache.lucene.codecs.lucene410;
/*
* 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.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
import org.apache.lucene.codecs.lucene49.Lucene49RWNormsFormat;
/**
* Read-Write version of 4.10 codec for testing
* @deprecated for test purposes only
*/
@Deprecated
public final class Lucene410RWCodec extends Lucene410Codec {
private static final DocValuesFormat docValues = new Lucene410RWDocValuesFormat();
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return docValues;
}
private static final NormsFormat norms = new Lucene49RWNormsFormat();
@Override
public NormsFormat normsFormat() {
return norms;
}
private static final SegmentInfoFormat segmentInfos = new Lucene46RWSegmentInfoFormat();
@Override
public SegmentInfoFormat segmentInfoFormat() {
return segmentInfos;
}
private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
@Override
public StoredFieldsFormat storedFieldsFormat() {
return storedFields;
}
private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
@Override
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
}

View File

@ -0,0 +1,42 @@
package org.apache.lucene.codecs.lucene410;
/*
* 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.DocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentWriteState;
/**
* Read-Write version of 4.10 docvalues format for testing
* @deprecated for test purposes only
*/
class Lucene410RWDocValuesFormat extends Lucene410DocValuesFormat {
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene410DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION) {
@Override
void checkCanWrite(FieldInfo field) {
// allow writing all fields
}
};
}
}

View File

@ -51,7 +51,7 @@ import org.apache.lucene.util.TestUtil;
* Tests Lucene410DocValuesFormat
*/
public class TestLucene410DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene410DocValuesFormat());
private final Codec codec = new Lucene410RWCodec();
@Override
protected Codec getCodec() {
@ -132,7 +132,7 @@ public class TestLucene410DocValuesFormat extends BaseCompressingDocValuesFormat
// break;
default: throw new AssertionError();
}
final DocValuesFormat dv = new Lucene410DocValuesFormat();
final DocValuesFormat dv = new Lucene410RWDocValuesFormat();
conf.setCodec(new AssertingCodec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {

View File

@ -31,8 +31,9 @@ import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
/**
* Read-write version of {@link Lucene45Codec} for testing.
* @deprecated for test purposes only
*/
@SuppressWarnings("deprecation")
@Deprecated
public final class Lucene45RWCodec extends Lucene45Codec {
private static final FieldInfosFormat fieldInfosFormat = new Lucene42RWFieldInfosFormat();

View File

@ -32,7 +32,10 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
/**
* Tests performing docvalues updates against versions of lucene
* that did not support it.
*/
public class TestDocValuesUpdatesOnOldSegments extends LuceneTestCase {
static long getValue(BinaryDocValues bdv, int idx) {
@ -60,56 +63,62 @@ public class TestDocValuesUpdatesOnOldSegments extends LuceneTestCase {
public void testBinaryUpdates() throws Exception {
Codec[] oldCodecs = new Codec[] { new Lucene40RWCodec(), new Lucene41RWCodec(), new Lucene42RWCodec(), new Lucene45RWCodec() };
Directory dir = newDirectory();
// create a segment with an old Codec
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setCodec(oldCodecs[random().nextInt(oldCodecs.length)]);
IndexWriter writer = new IndexWriter(dir, conf);
Document doc = new Document();
doc.add(new StringField("id", "doc", Store.NO));
doc.add(new BinaryDocValuesField("f", toBytes(5L)));
writer.addDocument(doc);
writer.close();
conf = newIndexWriterConfig(new MockAnalyzer(random()));
writer = new IndexWriter(dir, conf);
writer.updateBinaryDocValue(new Term("id", "doc"), "f", toBytes(4L));
try {
for (Codec codec : oldCodecs) {
Directory dir = newDirectory();
// create a segment with an old Codec
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setCodec(codec);
IndexWriter writer = new IndexWriter(dir, conf);
Document doc = new Document();
doc.add(new StringField("id", "doc", Store.NO));
doc.add(new BinaryDocValuesField("f", toBytes(5L)));
writer.addDocument(doc);
writer.close();
fail("should not have succeeded to update a segment written with an old Codec");
} catch (UnsupportedOperationException e) {
writer.rollback();
conf = newIndexWriterConfig(new MockAnalyzer(random()));
writer = new IndexWriter(dir, conf);
writer.updateBinaryDocValue(new Term("id", "doc"), "f", toBytes(4L));
try {
writer.close();
fail("should not have succeeded to update a segment written with an old Codec");
} catch (UnsupportedOperationException e) {
writer.rollback();
}
dir.close();
}
dir.close();
}
public void testNumericUpdates() throws Exception {
Codec[] oldCodecs = new Codec[] { new Lucene40RWCodec(), new Lucene41RWCodec(), new Lucene42RWCodec(), new Lucene45RWCodec() };
Directory dir = newDirectory();
// create a segment with an old Codec
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setCodec(oldCodecs[random().nextInt(oldCodecs.length)]);
IndexWriter writer = new IndexWriter(dir, conf);
Document doc = new Document();
doc.add(new StringField("id", "doc", Store.NO));
doc.add(new NumericDocValuesField("f", 5));
writer.addDocument(doc);
writer.close();
conf = newIndexWriterConfig(new MockAnalyzer(random()));
writer = new IndexWriter(dir, conf);
writer.updateNumericDocValue(new Term("id", "doc"), "f", 4L);
try {
for (Codec codec : oldCodecs) {
Directory dir = newDirectory();
// create a segment with an old Codec
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setCodec(codec);
IndexWriter writer = new IndexWriter(dir, conf);
Document doc = new Document();
doc.add(new StringField("id", "doc", Store.NO));
doc.add(new NumericDocValuesField("f", 5));
writer.addDocument(doc);
writer.close();
fail("should not have succeeded to update a segment written with an old Codec");
} catch (UnsupportedOperationException e) {
writer.rollback();
conf = newIndexWriterConfig(new MockAnalyzer(random()));
writer = new IndexWriter(dir, conf);
writer.updateNumericDocValue(new Term("id", "doc"), "f", 4L);
try {
writer.close();
fail("should not have succeeded to update a segment written with an old Codec");
} catch (UnsupportedOperationException e) {
writer.rollback();
}
dir.close();
}
dir.close();
}
}

View File

@ -1,25 +0,0 @@
<!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 4.10 file format.
</body>
</html>

View File

@ -115,7 +115,7 @@ public class Lucene50Codec extends Codec {
/** Returns the docvalues format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene410"
* The default implementation always returns "Lucene50"
*/
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
@ -127,7 +127,7 @@ public class Lucene50Codec extends Codec {
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene410");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
private final NormsFormat normsFormat = new Lucene50NormsFormat();

View File

@ -0,0 +1,586 @@
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.Closeable; // javadocs
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MathUtil;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.packed.DirectWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
/** writer for {@link Lucene50DocValuesFormat} */
class Lucene50DocValuesConsumer extends DocValuesConsumer implements Closeable {
static final int BLOCK_SIZE = 16384;
// address terms in blocks of 16 terms
static final int INTERVAL_SHIFT = 4;
static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
// build reverse index from every 1024th term
static final int REVERSE_INTERVAL_SHIFT = 10;
static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
// for conversion from reverse index to block
static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
/** Compressed using packed blocks of ints. */
public static final int DELTA_COMPRESSED = 0;
/** Compressed by computing the GCD. */
public static final int GCD_COMPRESSED = 1;
/** Compressed by giving IDs to unique values. */
public static final int TABLE_COMPRESSED = 2;
/** Compressed with monotonically increasing values */
public static final int MONOTONIC_COMPRESSED = 3;
/** Compressed with constant value (uses only missing bitset) */
public static final int CONST_COMPRESSED = 4;
/** Uncompressed binary, written directly (fixed length). */
public static final int BINARY_FIXED_UNCOMPRESSED = 0;
/** Uncompressed binary, written directly (variable length). */
public static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
/** Compressed binary with shared prefixes */
public static final int BINARY_PREFIX_COMPRESSED = 2;
/** Standard storage for sorted set values with 1 level of indirection:
* docId -> address -> ord. */
public static final int SORTED_WITH_ADDRESSES = 0;
/** Single-valued sorted set values, encoded as sorted values, so no level
* of indirection: docId -> ord. */
public static final int SORTED_SINGLE_VALUED = 1;
/** placeholder for missing offset that means there are no missing values */
public static final int ALL_LIVE = -1;
/** placeholder for missing offset that means all values are missing */
public static final int ALL_MISSING = -2;
IndexOutput data, meta;
final int maxDoc;
/** expert: Creates a new writer */
public Lucene50DocValuesConsumer(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, Lucene50DocValuesFormat.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.writeSegmentHeader(meta, metaCodec, Lucene50DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
maxDoc = state.segmentInfo.getDocCount();
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this);
}
}
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
addNumericField(field, values, true);
}
void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
long count = 0;
long minValue = Long.MAX_VALUE;
long maxValue = Long.MIN_VALUE;
long gcd = 0;
long missingCount = 0;
long zeroCount = 0;
// TODO: more efficient?
HashSet<Long> uniqueValues = null;
if (optimizeStorage) {
uniqueValues = new HashSet<>();
for (Number nv : values) {
final long v;
if (nv == null) {
v = 0;
missingCount++;
zeroCount++;
} else {
v = nv.longValue();
if (v == 0) {
zeroCount++;
}
}
if (gcd != 1) {
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
// in that case v - minValue might overflow and make the GCD computation return
// wrong results. Since these extreme values are unlikely, we just discard
// GCD computation for them
gcd = 1;
} else if (count != 0) { // minValue needs to be set first
gcd = MathUtil.gcd(gcd, v - minValue);
}
}
minValue = Math.min(minValue, v);
maxValue = Math.max(maxValue, v);
if (uniqueValues != null) {
if (uniqueValues.add(v)) {
if (uniqueValues.size() > 256) {
uniqueValues = null;
}
}
}
++count;
}
} else {
for (Number nv : values) {
long v = nv.longValue();
minValue = Math.min(minValue, v);
maxValue = Math.max(maxValue, v);
++count;
}
}
final long delta = maxValue - minValue;
final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
final int tableBitsRequired = uniqueValues == null
? Integer.MAX_VALUE
: DirectWriter.bitsRequired(uniqueValues.size() - 1);
final int format;
if (uniqueValues != null
&& count <= Integer.MAX_VALUE
&& (uniqueValues.size() == 1
|| (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
// either one unique value C or two unique values: "missing" and C
format = CONST_COMPRESSED;
} else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
format = TABLE_COMPRESSED;
} else if (gcd != 0 && gcd != 1) {
final long gcdDelta = (maxValue - minValue) / gcd;
final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
} else {
format = DELTA_COMPRESSED;
}
meta.writeVInt(field.number);
meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
meta.writeVInt(format);
if (missingCount == 0) {
meta.writeLong(ALL_LIVE);
} else if (missingCount == count) {
meta.writeLong(ALL_MISSING);
} else {
meta.writeLong(data.getFilePointer());
writeMissingBitset(values);
}
meta.writeLong(data.getFilePointer());
meta.writeVLong(count);
switch (format) {
case CONST_COMPRESSED:
// write the constant (nonzero value in the n=2 case, singleton value otherwise)
meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
break;
case GCD_COMPRESSED:
meta.writeLong(minValue);
meta.writeLong(gcd);
final long maxDelta = (maxValue - minValue) / gcd;
final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
meta.writeVInt(bits);
final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
for (Number nv : values) {
long value = nv == null ? 0 : nv.longValue();
quotientWriter.add((value - minValue) / gcd);
}
quotientWriter.finish();
break;
case DELTA_COMPRESSED:
final long minDelta = delta < 0 ? 0 : minValue;
meta.writeLong(minDelta);
meta.writeVInt(deltaBitsRequired);
final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
for (Number nv : values) {
long v = nv == null ? 0 : nv.longValue();
writer.add(v - minDelta);
}
writer.finish();
break;
case TABLE_COMPRESSED:
final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
Arrays.sort(decode);
final HashMap<Long,Integer> encode = new HashMap<>();
meta.writeVInt(decode.length);
for (int i = 0; i < decode.length; i++) {
meta.writeLong(decode[i]);
encode.put(decode[i], i);
}
meta.writeVInt(tableBitsRequired);
final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
for (Number nv : values) {
ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
}
ordsWriter.finish();
break;
default:
throw new AssertionError();
}
meta.writeLong(data.getFilePointer());
}
// TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
// but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
void writeMissingBitset(Iterable<?> values) throws IOException {
byte bits = 0;
int count = 0;
for (Object v : values) {
if (count == 8) {
data.writeByte(bits);
count = 0;
bits = 0;
}
if (v != null) {
bits |= 1 << (count & 7);
}
count++;
}
if (count > 0) {
data.writeByte(bits);
}
}
@Override
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
// write the byte[] data
meta.writeVInt(field.number);
meta.writeByte(Lucene50DocValuesFormat.BINARY);
int minLength = Integer.MAX_VALUE;
int maxLength = Integer.MIN_VALUE;
final long startFP = data.getFilePointer();
long count = 0;
long missingCount = 0;
for(BytesRef v : values) {
final int length;
if (v == null) {
length = 0;
missingCount++;
} else {
length = v.length;
}
minLength = Math.min(minLength, length);
maxLength = Math.max(maxLength, length);
if (v != null) {
data.writeBytes(v.bytes, v.offset, v.length);
}
count++;
}
meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
if (missingCount == 0) {
meta.writeLong(ALL_LIVE);
} else if (missingCount == count) {
meta.writeLong(ALL_MISSING);
} else {
meta.writeLong(data.getFilePointer());
writeMissingBitset(values);
}
meta.writeVInt(minLength);
meta.writeVInt(maxLength);
meta.writeVLong(count);
meta.writeLong(startFP);
// if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
// otherwise, we need to record the length fields...
if (minLength != maxLength) {
meta.writeLong(data.getFilePointer());
meta.writeVInt(PackedInts.VERSION_CURRENT);
meta.writeVInt(BLOCK_SIZE);
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
long addr = 0;
writer.add(addr);
for (BytesRef v : values) {
if (v != null) {
addr += v.length;
}
writer.add(addr);
}
writer.finish();
}
}
/** expert: writes a value dictionary for a sorted/sortedset field */
private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
// first check if its a "fixed-length" terms dict
int minLength = Integer.MAX_VALUE;
int maxLength = Integer.MIN_VALUE;
long numValues = 0;
for (BytesRef v : values) {
minLength = Math.min(minLength, v.length);
maxLength = Math.max(maxLength, v.length);
numValues++;
}
if (minLength == maxLength) {
// no index needed: direct addressing by mult
addBinaryField(field, values);
} else if (numValues < REVERSE_INTERVAL_COUNT) {
// low cardinality: waste a few KB of ram, but can't really use fancy index etc
addBinaryField(field, values);
} else {
assert numValues > 0; // we don't have to handle the empty case
// header
meta.writeVInt(field.number);
meta.writeByte(Lucene50DocValuesFormat.BINARY);
meta.writeVInt(BINARY_PREFIX_COMPRESSED);
meta.writeLong(-1L);
// now write the bytes: sharing prefixes within a block
final long startFP = data.getFilePointer();
// currently, we have to store the delta from expected for every 1/nth term
// we could avoid this, but its not much and less overall RAM than the previous approach!
RAMOutputStream addressBuffer = new RAMOutputStream();
MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, BLOCK_SIZE);
// buffers up 16 terms
RAMOutputStream bytesBuffer = new RAMOutputStream();
// buffers up block header
RAMOutputStream headerBuffer = new RAMOutputStream();
BytesRefBuilder lastTerm = new BytesRefBuilder();
lastTerm.grow(maxLength);
long count = 0;
int suffixDeltas[] = new int[INTERVAL_COUNT];
for (BytesRef v : values) {
int termPosition = (int) (count & INTERVAL_MASK);
if (termPosition == 0) {
termAddresses.add(data.getFilePointer() - startFP);
// abs-encode first term
headerBuffer.writeVInt(v.length);
headerBuffer.writeBytes(v.bytes, v.offset, v.length);
lastTerm.copyBytes(v);
} else {
// prefix-code: we only share at most 255 characters, to encode the length as a single
// byte and have random access. Larger terms just get less compression.
int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
bytesBuffer.writeByte((byte) sharedPrefix);
bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
// we can encode one smaller, because terms are unique.
suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
}
count++;
// flush block
if ((count & INTERVAL_MASK) == 0) {
flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
}
}
// flush trailing crap
int leftover = (int) (count & INTERVAL_MASK);
if (leftover > 0) {
Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
}
final long indexStartFP = data.getFilePointer();
// write addresses of indexed terms
termAddresses.finish();
addressBuffer.writeTo(data);
addressBuffer = null;
termAddresses = null;
meta.writeVInt(minLength);
meta.writeVInt(maxLength);
meta.writeVLong(count);
meta.writeLong(startFP);
meta.writeLong(indexStartFP);
meta.writeVInt(PackedInts.VERSION_CURRENT);
meta.writeVInt(BLOCK_SIZE);
addReverseTermIndex(field, values, maxLength);
}
}
// writes term dictionary "block"
// first term is absolute encoded as vint length + bytes.
// lengths of subsequent N terms are encoded as either N bytes or N shorts.
// in the double-byte case, the first byte is indicated with -1.
// subsequent terms are encoded as byte suffixLength + bytes.
private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
boolean twoByte = false;
for (int i = 1; i < suffixDeltas.length; i++) {
if (suffixDeltas[i] > 254) {
twoByte = true;
}
}
if (twoByte) {
headerBuffer.writeByte((byte)255);
for (int i = 1; i < suffixDeltas.length; i++) {
headerBuffer.writeShort((short) suffixDeltas[i]);
}
} else {
for (int i = 1; i < suffixDeltas.length; i++) {
headerBuffer.writeByte((byte) suffixDeltas[i]);
}
}
headerBuffer.writeTo(data);
headerBuffer.reset();
bytesBuffer.writeTo(data);
bytesBuffer.reset();
}
// writes reverse term index: used for binary searching a term into a range of 64 blocks
// for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
// terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
long count = 0;
BytesRefBuilder priorTerm = new BytesRefBuilder();
priorTerm.grow(maxLength);
BytesRef indexTerm = new BytesRef();
long startFP = data.getFilePointer();
PagedBytes pagedBytes = new PagedBytes(15);
MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
for (BytesRef b : values) {
int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
if (termPosition == 0) {
int len = StringHelper.sortKeyLength(priorTerm.get(), b);
indexTerm.bytes = b.bytes;
indexTerm.offset = b.offset;
indexTerm.length = len;
addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
} else if (termPosition == REVERSE_INTERVAL_MASK) {
priorTerm.copyBytes(b);
}
count++;
}
addresses.finish();
long numBytes = pagedBytes.getPointer();
pagedBytes.freeze(true);
PagedBytesDataInput in = pagedBytes.getDataInput();
meta.writeLong(startFP);
data.writeVLong(numBytes);
data.copyBytes(in, numBytes);
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene50DocValuesFormat.SORTED);
addTermsDict(field, values);
addNumericField(field, docToOrd, false);
}
@Override
public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene50DocValuesFormat.SORTED_NUMERIC);
if (isSingleValued(docToValueCount)) {
meta.writeVInt(SORTED_SINGLE_VALUED);
// The field is single-valued, we can encode it as NUMERIC
addNumericField(field, singletonView(docToValueCount, values, null));
} else {
meta.writeVInt(SORTED_WITH_ADDRESSES);
// write the stream of values as a numeric field
addNumericField(field, values, true);
// write the doc -> ord count as a absolute index to the stream
addAddresses(field, docToValueCount);
}
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene50DocValuesFormat.SORTED_SET);
if (isSingleValued(docToOrdCount)) {
meta.writeVInt(SORTED_SINGLE_VALUED);
// The field is single-valued, we can encode it as SORTED
addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
} else {
meta.writeVInt(SORTED_WITH_ADDRESSES);
// write the ord -> byte[] as a binary field
addTermsDict(field, values);
// write the stream of ords as a numeric field
// NOTE: we could return an iterator that delta-encodes these within a doc
addNumericField(field, ords, false);
// write the doc -> ord count as a absolute index to the stream
addAddresses(field, docToOrdCount);
}
}
// writes addressing information as MONOTONIC_COMPRESSED integer
private void addAddresses(FieldInfo field, Iterable<Number> values) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
meta.writeVInt(MONOTONIC_COMPRESSED);
meta.writeLong(-1L);
meta.writeLong(data.getFilePointer());
meta.writeVLong(maxDoc);
meta.writeVInt(PackedInts.VERSION_CURRENT);
meta.writeVInt(BLOCK_SIZE);
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
long addr = 0;
writer.add(addr);
for (Number v : values) {
addr += v.longValue();
writer.add(addr);
}
writer.finish();
meta.writeLong(data.getFilePointer());
}
@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;
}
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene410;
package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -33,7 +33,7 @@ import org.apache.lucene.util.packed.DirectWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
/**
* Lucene 4.10 DocValues format.
* Lucene 5.0 DocValues format.
* <p>
* Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
* <p>
@ -49,6 +49,8 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
* common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
* <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are written
* as blocks of bitpacked integers, encoding the deviation from the expected delta.
* <li>Const-compressed: when there is only one possible non-missing value, only the missing
* bitset is encoded.
* </ul>
* <p>
* {@link DocValuesType#BINARY BINARY}:
@ -145,7 +147,7 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
* Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
* is written for the addresses.
* <p>MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
* If its -1, then there are no missing values.
* If its -1, then there are no missing values. If its -2, all values are missing.
* <p>Checksum contains the CRC32 checksum of all bytes in the .dvm file up
* until the checksum. This is used to verify integrity of the file on opening the
* index.
@ -164,26 +166,26 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
* </ol>
* @lucene.experimental
*/
public final class Lucene410DocValuesFormat extends DocValuesFormat {
public final class Lucene50DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */
public Lucene410DocValuesFormat() {
super("Lucene410");
public Lucene50DocValuesFormat() {
super("Lucene50");
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene410DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
return new Lucene50DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene410DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
return new Lucene50DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
static final String DATA_CODEC = "Lucene410DocValuesData";
static final String DATA_CODEC = "Lucene50DocValuesData";
static final String DATA_EXTENSION = "dvd";
static final String META_CODEC = "Lucene410ValuesMetadata";
static final String META_CODEC = "Lucene50ValuesMetadata";
static final String META_EXTENSION = "dvm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;

View File

@ -185,7 +185,7 @@ term frequency. To add Term Vectors to your index see the
{@link org.apache.lucene.document.Field Field} constructors
</li>
<li>
{@link org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat Per-document values}.
{@link org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat Per-document values}.
Like stored values, these are also keyed by document
number, but are generally intended to be loaded into main memory for fast
access. Whereas stored values are generally intended for summary results from
@ -294,7 +294,7 @@ systems that frequently run out of file handles.</td>
<td>Encodes length and boost factors for docs and fields</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat Per-Document Values}</td>
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat Per-Document Values}</td>
<td>.dvd, .dvm</td>
<td>Encodes additional scoring factors or other per-document information.</td>
</tr>

View File

@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat
org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat

View File

@ -0,0 +1,285 @@
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.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.blocktreeords.Ords41PostingsFormat;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
/**
* Tests Lucene50DocValuesFormat
*/
public class TestLucene50DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene50DocValuesFormat());
@Override
protected Codec getCodec() {
return codec;
}
// TODO: these big methods can easily blow up some of the other ram-hungry codecs...
// for now just keep them here, as we want to test this for this format.
public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16);
}
}
@Nightly
public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16);
}
}
public void testSortedVariableLengthBigVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedVsStoredFields(atLeast(300), 1, 32766);
}
}
@Nightly
public void testSortedVariableLengthManyVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
}
}
public void testTermsEnumFixedWidth() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
}
}
public void testTermsEnumVariableWidth() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
}
}
@Nightly
public void testTermsEnumRandomMany() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
}
}
// TODO: try to refactor this and some termsenum tests into the base class.
// to do this we need to fix the test class to get a DVF not a Codec so we can setup
// the postings format correctly.
private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
Directory dir = newFSDirectory(createTempDir());
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMergeScheduler(new SerialMergeScheduler());
// set to duel against a codec which has ordinals:
final PostingsFormat pf;
switch (random().nextInt(2)) {
case 0: pf = new Lucene41WithOrds();
break;
case 1: pf = new Ords41PostingsFormat();
break;
// TODO: these don't actually support ords!
//case 2: pf = new FSTOrdPostingsFormat();
// break;
default: throw new AssertionError();
}
final DocValuesFormat dv = new Lucene50DocValuesFormat();
conf.setCodec(new AssertingCodec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return pf;
}
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return dv;
}
});
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
// index some docs
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
doc.add(idField);
final int length = TestUtil.nextInt(random(), minLength, maxLength);
int numValues = random().nextInt(17);
// create a random list of strings
List<String> values = new ArrayList<>();
for (int v = 0; v < numValues; v++) {
values.add(TestUtil.randomSimpleString(random(), minLength, length));
}
// add in any order to the indexed field
ArrayList<String> unordered = new ArrayList<>(values);
Collections.shuffle(unordered, random());
for (String v : values) {
doc.add(newStringField("indexed", v, Field.Store.NO));
}
// add in any order to the dv field
ArrayList<String> unordered2 = new ArrayList<>(values);
Collections.shuffle(unordered2, random());
for (String v : unordered2) {
doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
}
writer.addDocument(doc);
if (random().nextInt(31) == 0) {
writer.commit();
}
}
// delete some docs
int numDeletions = random().nextInt(numDocs/10);
for (int i = 0; i < numDeletions; i++) {
int id = random().nextInt(numDocs);
writer.deleteDocuments(new Term("id", Integer.toString(id)));
}
// compare per-segment
DirectoryReader ir = writer.getReader();
for (LeafReaderContext context : ir.leaves()) {
LeafReader r = context.reader();
Terms terms = r.terms("indexed");
if (terms != null) {
assertEquals(terms.size(), r.getSortedSetDocValues("dv").getValueCount());
TermsEnum expected = terms.iterator(null);
TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
assertEquals(terms.size(), expected, actual);
}
}
ir.close();
writer.forceMerge(1);
// now compare again after the merge
ir = writer.getReader();
LeafReader ar = getOnlySegmentReader(ir);
Terms terms = ar.terms("indexed");
if (terms != null) {
assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
TermsEnum expected = terms.iterator(null);
TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
assertEquals(terms.size(), expected, actual);
}
ir.close();
writer.close();
dir.close();
}
private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
BytesRef ref;
// sequential next() through all terms
while ((ref = expected.next()) != null) {
assertEquals(ref, actual.next());
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
assertNull(actual.next());
// sequential seekExact(ord) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
actual.seekExact(i);
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// sequential seekExact(BytesRef) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
assertTrue(actual.seekExact(expected.term()));
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// sequential seekCeil(BytesRef) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekExact(ord)
for (long i = 0; i < numOrds; i++) {
long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
expected.seekExact(randomOrd);
actual.seekExact(randomOrd);
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekExact(BytesRef)
for (long i = 0; i < numOrds; i++) {
long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
expected.seekExact(randomOrd);
actual.seekExact(expected.term());
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekCeil(BytesRef)
for (long i = 0; i < numOrds; i++) {
BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
SeekStatus expectedStatus = expected.seekCeil(target);
assertEquals(expectedStatus, actual.seekCeil(target));
if (expectedStatus != SeekStatus.END) {
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
}
}
}

View File

@ -2883,4 +2883,6 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
protected boolean codecSupportsSortedNumeric() {
return true;
}
// nocommit: ensure every type has a *mergeAwayAllValues test, and add simple constant tests
}

View File

@ -46,8 +46,8 @@ import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
import org.apache.lucene.codecs.lucene50.Lucene50Codec;
import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@ -761,7 +761,7 @@ public final class TestUtil {
* Returns the actual default docvalues format (e.g. LuceneMNDocValuesFormat for this version of Lucene.
*/
public static DocValuesFormat getDefaultDocValuesFormat() {
return new Lucene410DocValuesFormat();
return new Lucene50DocValuesFormat();
}
// TODO: generalize all 'test-checks-for-crazy-codecs' to