mirror of https://github.com/apache/lucene.git
LUCENE-7453: Create a Lucene70Codec.
This commit is contained in:
parent
2ad00826a3
commit
32446e9205
|
@ -29,7 +29,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
|
|||
import org.apache.lucene.benchmark.byTask.utils.Config;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
|
||||
import org.apache.lucene.index.ConcurrentMergeScheduler;
|
||||
import org.apache.lucene.index.IndexCommit;
|
||||
import org.apache.lucene.index.IndexDeletionPolicy;
|
||||
|
@ -139,7 +139,7 @@ public class CreateIndexTask extends PerfTask {
|
|||
if (defaultCodec == null && postingsFormat != null) {
|
||||
try {
|
||||
final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
|
||||
iwConf.setCodec(new Lucene62Codec() {
|
||||
iwConf.setCodec(new Lucene70Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return postingsFormatChosen;
|
||||
|
|
|
@ -57,7 +57,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
}
|
||||
|
||||
// TODO: should we use this, or maybe a system property is better?
|
||||
static Codec defaultCodec = LOADER.lookup("Lucene62");
|
||||
static Codec defaultCodec = LOADER.lookup("Lucene70");
|
||||
}
|
||||
|
||||
private final String name;
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -95,7 +96,9 @@ import org.apache.lucene.util.packed.DirectWriter;
|
|||
* <li><tt>.dvm</tt>: DocValues metadata</li>
|
||||
* </ol>
|
||||
* @lucene.experimental
|
||||
* @deprecated Use {@link Lucene70DocValuesFormat}.
|
||||
*/
|
||||
@Deprecated
|
||||
public final class Lucene54DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
/** Sole Constructor */
|
||||
|
|
|
@ -0,0 +1,177 @@
|
|||
/*
|
||||
* 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.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
|
||||
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;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 7.0 index format, with configurable per-field postings
|
||||
* and docvalues formats.
|
||||
* <p>
|
||||
* If you want to reuse functionality of this codec in another codec, extend
|
||||
* {@link FilterCodec}.
|
||||
*
|
||||
* @see org.apache.lucene.codecs.lucene70 package documentation for file format details.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Lucene70Codec extends Codec {
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
|
||||
private final SegmentInfoFormat segmentInfosFormat = new Lucene62SegmentInfoFormat();
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
|
||||
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
|
||||
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return Lucene70Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return Lucene70Codec.this.getDocValuesFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final StoredFieldsFormat storedFieldsFormat;
|
||||
|
||||
/**
|
||||
* Instantiates a new codec.
|
||||
*/
|
||||
public Lucene70Codec() {
|
||||
this(Mode.BEST_SPEED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Instantiates a new codec, specifying the stored fields compression
|
||||
* mode to use.
|
||||
* @param mode stored fields compression mode to use for newly
|
||||
* flushed/merged segments.
|
||||
*/
|
||||
public Lucene70Codec(Mode mode) {
|
||||
super("Lucene70");
|
||||
this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PostingsFormat postingsFormat() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final FieldInfosFormat fieldInfosFormat() {
|
||||
return fieldInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final LiveDocsFormat liveDocsFormat() {
|
||||
return liveDocsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final CompoundFormat compoundFormat() {
|
||||
return compoundFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PointsFormat pointsFormat() {
|
||||
return new Lucene60PointsFormat();
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene50".
|
||||
* <p>
|
||||
* <b>WARNING:</b> if you subclass, you are responsible for index
|
||||
* backwards compatibility: future version of Lucene are only
|
||||
* guaranteed to be able to read the default implementation.
|
||||
*/
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return defaultFormat;
|
||||
}
|
||||
|
||||
/** Returns the docvalues format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene70".
|
||||
* <p>
|
||||
* <b>WARNING:</b> if you subclass, you are responsible for index
|
||||
* backwards compatibility: future version of Lucene are only
|
||||
* guaranteed to be able to read the default implementation.
|
||||
*/
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return defaultDVFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70");
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene53NormsFormat();
|
||||
|
||||
@Override
|
||||
public final NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,797 @@
|
|||
/*
|
||||
* 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.Closeable; // javadocs
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.LegacyDocValuesIterables;
|
||||
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.LongsRef;
|
||||
import org.apache.lucene.util.MathUtil;
|
||||
import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.packed.DirectMonotonicWriter;
|
||||
import org.apache.lucene.util.packed.DirectWriter;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.*;
|
||||
|
||||
/** writer for {@link Lucene70DocValuesFormat} */
|
||||
final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
||||
|
||||
enum NumberType {
|
||||
/** Dense ordinals */
|
||||
ORDINAL,
|
||||
/** Random long values */
|
||||
VALUE;
|
||||
}
|
||||
|
||||
IndexOutput data, meta;
|
||||
final int maxDoc;
|
||||
|
||||
/** expert: Creates a new writer */
|
||||
public Lucene70DocValuesConsumer(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, Lucene70DocValuesFormat.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, Lucene70DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
||||
addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE);
|
||||
}
|
||||
|
||||
void addNumericField(FieldInfo field, Iterable<Number> values, NumberType numberType) 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;
|
||||
long missingOrdCount = 0;
|
||||
if (numberType == NumberType.VALUE) {
|
||||
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();
|
||||
if (v == -1L) {
|
||||
missingOrdCount++;
|
||||
}
|
||||
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 boolean sparse; // 10% of docs or less have a value
|
||||
switch (numberType) {
|
||||
case VALUE:
|
||||
sparse = (double) missingCount / count >= 0.90;
|
||||
break;
|
||||
case ORDINAL:
|
||||
sparse = (double) missingOrdCount / count >= 0.90;
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
|
||||
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 (sparse && count >= 1024) {
|
||||
// require at least 1024 docs to avoid flipping back and forth when doing NRT search
|
||||
format = SPARSE_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(Lucene70DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(format);
|
||||
if (format == SPARSE_COMPRESSED) {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
final long numDocsWithValue;
|
||||
switch (numberType) {
|
||||
case VALUE:
|
||||
numDocsWithValue = count - missingCount;
|
||||
break;
|
||||
case ORDINAL:
|
||||
numDocsWithValue = count - missingOrdCount;
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
final long maxDoc = writeSparseMissingBitset(values, numberType, numDocsWithValue);
|
||||
assert maxDoc == count;
|
||||
} else 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;
|
||||
case SPARSE_COMPRESSED:
|
||||
final Iterable<Number> filteredMissingValues;
|
||||
switch (numberType) {
|
||||
case VALUE:
|
||||
meta.writeByte((byte) 0);
|
||||
filteredMissingValues = new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return StreamSupport
|
||||
.stream(values.spliterator(), false)
|
||||
.filter(value -> value != null)
|
||||
.iterator();
|
||||
}
|
||||
};
|
||||
break;
|
||||
case ORDINAL:
|
||||
meta.writeByte((byte) 1);
|
||||
filteredMissingValues = new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return StreamSupport
|
||||
.stream(values.spliterator(), false)
|
||||
.filter(value -> value.longValue() != -1L)
|
||||
.iterator();
|
||||
}
|
||||
};
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
// Write non-missing values as a numeric field
|
||||
addNumericField(field, filteredMissingValues, numberType);
|
||||
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 {
|
||||
long bits = 0;
|
||||
int count = 0;
|
||||
for (Object v : values) {
|
||||
if (count == 64) {
|
||||
data.writeLong(bits);
|
||||
count = 0;
|
||||
bits = 0;
|
||||
}
|
||||
if (v != null) {
|
||||
bits |= 1L << count;
|
||||
}
|
||||
count++;
|
||||
}
|
||||
if (count > 0) {
|
||||
data.writeLong(bits);
|
||||
}
|
||||
}
|
||||
|
||||
long writeSparseMissingBitset(Iterable<Number> values, NumberType numberType, long numDocsWithValue) throws IOException {
|
||||
meta.writeVLong(numDocsWithValue);
|
||||
|
||||
// Write doc IDs that have a value
|
||||
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
|
||||
final DirectMonotonicWriter docIdsWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithValue, DIRECT_MONOTONIC_BLOCK_SHIFT);
|
||||
long docID = 0;
|
||||
for (Number nv : values) {
|
||||
switch (numberType) {
|
||||
case VALUE:
|
||||
if (nv != null) {
|
||||
docIdsWriter.add(docID);
|
||||
}
|
||||
break;
|
||||
case ORDINAL:
|
||||
if (nv.longValue() != -1L) {
|
||||
docIdsWriter.add(docID);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
docID++;
|
||||
}
|
||||
docIdsWriter.finish();
|
||||
return docID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
||||
addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc));
|
||||
}
|
||||
|
||||
private void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
// write the byte[] data
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene70DocValuesFormat.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, it's 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(DIRECT_MONOTONIC_BLOCK_SHIFT);
|
||||
|
||||
final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
|
||||
long addr = 0;
|
||||
writer.add(addr);
|
||||
for (BytesRef v : values) {
|
||||
if (v != null) {
|
||||
addr += v.length;
|
||||
}
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
meta.writeLong(data.getFilePointer());
|
||||
}
|
||||
}
|
||||
|
||||
/** expert: writes a value dictionary for a sorted/sortedset field */
|
||||
private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
|
||||
// first check if it's a "fixed-length" terms dict, and compressibility if so
|
||||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
long numValues = 0;
|
||||
BytesRefBuilder previousValue = new BytesRefBuilder();
|
||||
long prefixSum = 0; // only valid for fixed-width data, as we have a choice there
|
||||
for (BytesRef v : values) {
|
||||
minLength = Math.min(minLength, v.length);
|
||||
maxLength = Math.max(maxLength, v.length);
|
||||
if (minLength == maxLength) {
|
||||
int termPosition = (int) (numValues & INTERVAL_MASK);
|
||||
if (termPosition == 0) {
|
||||
// first term in block, save it away to compare against the last term later
|
||||
previousValue.copyBytes(v);
|
||||
} else if (termPosition == INTERVAL_COUNT - 1) {
|
||||
// last term in block, accumulate shared prefix against first term
|
||||
prefixSum += StringHelper.bytesDifference(previousValue.get(), v);
|
||||
}
|
||||
}
|
||||
numValues++;
|
||||
}
|
||||
// for fixed width data, look at the avg(shared prefix) before deciding how to encode:
|
||||
// prefix compression "costs" worst case 2 bytes per term because we must store suffix lengths.
|
||||
// so if we share at least 3 bytes on average, always compress.
|
||||
if (minLength == maxLength && prefixSum <= 3*(numValues >> INTERVAL_SHIFT)) {
|
||||
// no index needed: not very compressible, 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(Lucene70DocValuesFormat.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 it's not much and less overall RAM than the previous approach!
|
||||
RAMOutputStream addressBuffer = new RAMOutputStream();
|
||||
MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_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(MONOTONIC_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, MONOTONIC_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, DocValuesProducer valuesProducer) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene70DocValuesFormat.SORTED);
|
||||
addTermsDict(field, LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)));
|
||||
addNumericField(field, LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc), NumberType.ORDINAL);
|
||||
}
|
||||
|
||||
private void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> ords) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene70DocValuesFormat.SORTED);
|
||||
addTermsDict(field, values);
|
||||
addNumericField(field, ords, NumberType.ORDINAL);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException {
|
||||
|
||||
final Iterable<Number> docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc);
|
||||
final Iterable<Number> values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field);
|
||||
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene70DocValuesFormat.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), NumberType.VALUE);
|
||||
} else {
|
||||
final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
|
||||
if (uniqueValueSets != null) {
|
||||
meta.writeVInt(SORTED_SET_TABLE);
|
||||
|
||||
// write the set_id -> values mapping
|
||||
writeDictionary(uniqueValueSets);
|
||||
|
||||
// write the doc -> set_id as a numeric field
|
||||
addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), NumberType.ORDINAL);
|
||||
} else {
|
||||
meta.writeVInt(SORTED_WITH_ADDRESSES);
|
||||
// write the stream of values as a numeric field
|
||||
addNumericField(field, values, NumberType.VALUE);
|
||||
// write the doc -> ord count as a absolute index to the stream
|
||||
addOrdIndex(field, docToValueCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
||||
|
||||
Iterable<BytesRef> values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field));
|
||||
Iterable<Number> docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc);
|
||||
Iterable<Number> ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field);
|
||||
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene70DocValuesFormat.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 {
|
||||
final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
|
||||
if (uniqueValueSets != null) {
|
||||
meta.writeVInt(SORTED_SET_TABLE);
|
||||
|
||||
// write the set_id -> ords mapping
|
||||
writeDictionary(uniqueValueSets);
|
||||
|
||||
// write the ord -> byte[] as a binary field
|
||||
addTermsDict(field, values);
|
||||
|
||||
// write the doc -> set_id as a numeric field
|
||||
addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), NumberType.ORDINAL);
|
||||
} 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, NumberType.ORDINAL);
|
||||
|
||||
// write the doc -> ord count as a absolute index to the stream
|
||||
addOrdIndex(field, docToOrdCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
|
||||
Set<LongsRef> uniqueValueSet = new HashSet<>();
|
||||
LongsRef docValues = new LongsRef(256);
|
||||
|
||||
Iterator<Number> valueCountIterator = docToValueCount.iterator();
|
||||
Iterator<Number> valueIterator = values.iterator();
|
||||
int totalDictSize = 0;
|
||||
while (valueCountIterator.hasNext()) {
|
||||
docValues.length = valueCountIterator.next().intValue();
|
||||
if (docValues.length > 256) {
|
||||
return null;
|
||||
}
|
||||
for (int i = 0; i < docValues.length; ++i) {
|
||||
docValues.longs[i] = valueIterator.next().longValue();
|
||||
}
|
||||
if (uniqueValueSet.contains(docValues)) {
|
||||
continue;
|
||||
}
|
||||
totalDictSize += docValues.length;
|
||||
if (totalDictSize > 256) {
|
||||
return null;
|
||||
}
|
||||
uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
|
||||
}
|
||||
assert valueIterator.hasNext() == false;
|
||||
return new TreeSet<>(uniqueValueSet);
|
||||
}
|
||||
|
||||
private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
|
||||
int lengthSum = 0;
|
||||
for (LongsRef longs : uniqueValueSets) {
|
||||
lengthSum += longs.length;
|
||||
}
|
||||
|
||||
meta.writeInt(lengthSum);
|
||||
for (LongsRef valueSet : uniqueValueSets) {
|
||||
for (int i = 0; i < valueSet.length; ++i) {
|
||||
meta.writeLong(valueSet.longs[valueSet.offset + i]);
|
||||
}
|
||||
}
|
||||
|
||||
meta.writeInt(uniqueValueSets.size());
|
||||
for (LongsRef valueSet : uniqueValueSets) {
|
||||
meta.writeInt(valueSet.length);
|
||||
}
|
||||
}
|
||||
|
||||
private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
|
||||
final Map<LongsRef, Integer> setIds = new HashMap<>();
|
||||
int i = 0;
|
||||
for (LongsRef set : uniqueValueSets) {
|
||||
setIds.put(set, i++);
|
||||
}
|
||||
assert i == uniqueValueSets.size();
|
||||
|
||||
return new Iterable<Number>() {
|
||||
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
final Iterator<Number> valueCountIterator = docToValueCount.iterator();
|
||||
final Iterator<Number> valueIterator = values.iterator();
|
||||
final LongsRef docValues = new LongsRef(256);
|
||||
return new Iterator<Number>() {
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return valueCountIterator.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number next() {
|
||||
docValues.length = valueCountIterator.next().intValue();
|
||||
for (int i = 0; i < docValues.length; ++i) {
|
||||
docValues.longs[i] = valueIterator.next().longValue();
|
||||
}
|
||||
final Integer id = setIds.get(docValues);
|
||||
assert id != null;
|
||||
return id;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// writes addressing information as MONOTONIC_COMPRESSED integer
|
||||
private void addOrdIndex(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene70DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(MONOTONIC_COMPRESSED);
|
||||
meta.writeLong(-1L);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVLong(maxDoc);
|
||||
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
|
||||
|
||||
final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,183 @@
|
|||
/*
|
||||
* 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.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.SmallFloat;
|
||||
import org.apache.lucene.util.packed.DirectWriter;
|
||||
|
||||
/**
|
||||
* Lucene 7.0 DocValues format.
|
||||
* <p>
|
||||
* Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
|
||||
* <p>
|
||||
* {@link DocValuesType#NUMERIC NUMERIC}:
|
||||
* <ul>
|
||||
* <li>Delta-compressed: per-document integers written as deltas from the minimum value,
|
||||
* compressed with bitpacking. For more information, see {@link DirectWriter}.
|
||||
* <li>Table-compressed: when the number of unique values is very small (< 256), and
|
||||
* when there are unused "gaps" in the range of values used (such as {@link SmallFloat}),
|
||||
* a lookup table is written instead. Each per-document entry is instead the ordinal
|
||||
* to this table, and those ordinals are compressed with bitpacking ({@link DirectWriter}).
|
||||
* <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
|
||||
* 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.
|
||||
* <li>Sparse-compressed: only documents with a value are stored, and lookups are performed
|
||||
* using binary search.
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#BINARY BINARY}:
|
||||
* <ul>
|
||||
* <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
|
||||
* Each document's value can be addressed directly with multiplication ({@code docID * length}).
|
||||
* <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
|
||||
* for each document. The addresses are written as Monotonic-compressed numerics.
|
||||
* <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
|
||||
* completely and other values sharing prefixes. chunk addresses are written as Monotonic-compressed
|
||||
* numerics. A reverse lookup index is written from a portion of every 1024th term.
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#SORTED SORTED}:
|
||||
* <ul>
|
||||
* <li>Sorted: a mapping of ordinals to deduplicated terms is written as Binary,
|
||||
* along with the per-document ordinals written using one of the numeric strategies above.
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#SORTED_SET SORTED_SET}:
|
||||
* <ul>
|
||||
* <li>Single: if all documents have 0 or 1 value, then data are written like SORTED.
|
||||
* <li>SortedSet table: when there are few unique sets of values (< 256) then each set is assigned
|
||||
* an id, a lookup table is written and the mapping from document to set id is written using the
|
||||
* numeric strategies above.
|
||||
* <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Binary,
|
||||
* an ordinal list and per-document index into this list are written using the numeric strategies
|
||||
* above.
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
|
||||
* <ul>
|
||||
* <li>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
|
||||
* <li>SortedSet table: when there are few unique sets of values (< 256) then each set is assigned
|
||||
* an id, a lookup table is written and the mapping from document to set id is written using the
|
||||
* numeric strategies above.
|
||||
* <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
|
||||
* strategies above.
|
||||
* </ul>
|
||||
* <p>
|
||||
* Files:
|
||||
* <ol>
|
||||
* <li><tt>.dvd</tt>: DocValues data</li>
|
||||
* <li><tt>.dvm</tt>: DocValues metadata</li>
|
||||
* </ol>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene70DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
/** Sole Constructor */
|
||||
public Lucene70DocValuesFormat() {
|
||||
super("Lucene70");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new Lucene70DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return new Lucene70DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
static final String DATA_CODEC = "Lucene54DocValuesData";
|
||||
static final String DATA_EXTENSION = "dvd";
|
||||
static final String META_CODEC = "Lucene54DocValuesMetadata";
|
||||
static final String META_EXTENSION = "dvm";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
// indicates docvalues type
|
||||
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;
|
||||
|
||||
// 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. */
|
||||
static final int DELTA_COMPRESSED = 0;
|
||||
/** Compressed by computing the GCD. */
|
||||
static final int GCD_COMPRESSED = 1;
|
||||
/** Compressed by giving IDs to unique values. */
|
||||
static final int TABLE_COMPRESSED = 2;
|
||||
/** Compressed with monotonically increasing values */
|
||||
static final int MONOTONIC_COMPRESSED = 3;
|
||||
/** Compressed with constant value (uses only missing bitset) */
|
||||
static final int CONST_COMPRESSED = 4;
|
||||
/** Compressed with sparse arrays. */
|
||||
static final int SPARSE_COMPRESSED = 5;
|
||||
|
||||
/** Uncompressed binary, written directly (fixed length). */
|
||||
static final int BINARY_FIXED_UNCOMPRESSED = 0;
|
||||
/** Uncompressed binary, written directly (variable length). */
|
||||
static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
|
||||
/** Compressed binary with shared prefixes */
|
||||
static final int BINARY_PREFIX_COMPRESSED = 2;
|
||||
|
||||
/** Standard storage for sorted set values with 1 level of indirection:
|
||||
* {@code docId -> address -> ord}. */
|
||||
static final int SORTED_WITH_ADDRESSES = 0;
|
||||
/** Single-valued sorted set values, encoded as sorted values, so no level
|
||||
* of indirection: {@code docId -> ord}. */
|
||||
static final int SORTED_SINGLE_VALUED = 1;
|
||||
/** Compressed giving IDs to unique sets of values:
|
||||
* {@code docId -> setId -> ords} */
|
||||
static final int SORTED_SET_TABLE = 2;
|
||||
|
||||
/** placeholder for missing offset that means there are no missing values */
|
||||
static final int ALL_LIVE = -1;
|
||||
/** placeholder for missing offset that means all values are missing */
|
||||
static final int ALL_MISSING = -2;
|
||||
|
||||
// addressing uses 16k blocks
|
||||
static final int MONOTONIC_BLOCK_SIZE = 16384;
|
||||
static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,392 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Lucene 7.0 file format.
|
||||
*
|
||||
* <h1>Apache Lucene - Index File Formats</h1>
|
||||
* <div>
|
||||
* <ul>
|
||||
* <li><a href="#Introduction">Introduction</a></li>
|
||||
* <li><a href="#Definitions">Definitions</a>
|
||||
* <ul>
|
||||
* <li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
|
||||
* <li><a href="#Types_of_Fields">Types of Fields</a></li>
|
||||
* <li><a href="#Segments">Segments</a></li>
|
||||
* <li><a href="#Document_Numbers">Document Numbers</a></li>
|
||||
* </ul>
|
||||
* </li>
|
||||
* <li><a href="#Overview">Index Structure Overview</a></li>
|
||||
* <li><a href="#File_Naming">File Naming</a></li>
|
||||
* <li><a href="#file-names">Summary of File Extensions</a>
|
||||
* <ul>
|
||||
* <li><a href="#Lock_File">Lock File</a></li>
|
||||
* <li><a href="#History">History</a></li>
|
||||
* <li><a href="#Limitations">Limitations</a></li>
|
||||
* </ul>
|
||||
* </li>
|
||||
* </ul>
|
||||
* </div>
|
||||
* <a name="Introduction"></a>
|
||||
* <h2>Introduction</h2>
|
||||
* <div>
|
||||
* <p>This document defines the index file formats used in this version of Lucene.
|
||||
* If you are using a different version of Lucene, please consult the copy of
|
||||
* <code>docs/</code> that was distributed with
|
||||
* the version you are using.</p>
|
||||
* <p>This document attempts to provide a high-level definition of the Apache
|
||||
* Lucene file formats.</p>
|
||||
* </div>
|
||||
* <a name="Definitions"></a>
|
||||
* <h2>Definitions</h2>
|
||||
* <div>
|
||||
* <p>The fundamental concepts in Lucene are index, document, field and term.</p>
|
||||
* <p>An index contains a sequence of documents.</p>
|
||||
* <ul>
|
||||
* <li>A document is a sequence of fields.</li>
|
||||
* <li>A field is a named sequence of terms.</li>
|
||||
* <li>A term is a sequence of bytes.</li>
|
||||
* </ul>
|
||||
* <p>The same sequence of bytes in two different fields is considered a different
|
||||
* term. Thus terms are represented as a pair: the string naming the field, and the
|
||||
* bytes within the field.</p>
|
||||
* <a name="Inverted_Indexing"></a>
|
||||
* <h3>Inverted Indexing</h3>
|
||||
* <p>The index stores statistics about terms in order to make term-based search
|
||||
* more efficient. Lucene's index falls into the family of indexes known as an
|
||||
* <i>inverted index.</i> This is because it can list, for a term, the documents
|
||||
* that contain it. This is the inverse of the natural relationship, in which
|
||||
* documents list terms.</p>
|
||||
* <a name="Types_of_Fields"></a>
|
||||
* <h3>Types of Fields</h3>
|
||||
* <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
|
||||
* in the index literally, in a non-inverted manner. Fields that are inverted are
|
||||
* called <i>indexed</i>. A field may be both stored and indexed.</p>
|
||||
* <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
|
||||
* text of a field may be used literally as a term to be indexed. Most fields are
|
||||
* tokenized, but sometimes it is useful for certain identifier fields to be
|
||||
* indexed literally.</p>
|
||||
* <p>See the {@link org.apache.lucene.document.Field Field}
|
||||
* java docs for more information on Fields.</p>
|
||||
* <a name="Segments"></a>
|
||||
* <h3>Segments</h3>
|
||||
* <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
|
||||
* Each segment is a fully independent index, which could be searched separately.
|
||||
* Indexes evolve by:</p>
|
||||
* <ol>
|
||||
* <li>Creating new segments for newly added documents.</li>
|
||||
* <li>Merging existing segments.</li>
|
||||
* </ol>
|
||||
* <p>Searches may involve multiple segments and/or multiple indexes, each index
|
||||
* potentially composed of a set of segments.</p>
|
||||
* <a name="Document_Numbers"></a>
|
||||
* <h3>Document Numbers</h3>
|
||||
* <p>Internally, Lucene refers to documents by an integer <i>document number</i>.
|
||||
* The first document added to an index is numbered zero, and each subsequent
|
||||
* document added gets a number one greater than the previous.</p>
|
||||
* <p>Note that a document's number may change, so caution should be taken when
|
||||
* storing these numbers outside of Lucene. In particular, numbers may change in
|
||||
* the following situations:</p>
|
||||
* <ul>
|
||||
* <li>
|
||||
* <p>The numbers stored in each segment are unique only within the segment, and
|
||||
* must be converted before they can be used in a larger context. The standard
|
||||
* technique is to allocate each segment a range of values, based on the range of
|
||||
* numbers used in that segment. To convert a document number from a segment to an
|
||||
* external value, the segment's <i>base</i> document number is added. To convert
|
||||
* an external value back to a segment-specific value, the segment is identified
|
||||
* by the range that the external value is in, and the segment's base value is
|
||||
* subtracted. For example two five document segments might be combined, so that
|
||||
* the first segment has a base value of zero, and the second of five. Document
|
||||
* three from the second segment would have an external value of eight.</p>
|
||||
* </li>
|
||||
* <li>
|
||||
* <p>When documents are deleted, gaps are created in the numbering. These are
|
||||
* eventually removed as the index evolves through merging. Deleted documents are
|
||||
* dropped when segments are merged. A freshly-merged segment thus has no gaps in
|
||||
* its numbering.</p>
|
||||
* </li>
|
||||
* </ul>
|
||||
* </div>
|
||||
* <a name="Overview"></a>
|
||||
* <h2>Index Structure Overview</h2>
|
||||
* <div>
|
||||
* <p>Each segment index maintains the following:</p>
|
||||
* <ul>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment info}.
|
||||
* This contains metadata about a segment, such as the number of documents,
|
||||
* what files it uses,
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}.
|
||||
* This contains the set of field names used in the index.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}.
|
||||
* This contains, for each document, a list of attribute-value pairs, where the attributes
|
||||
* are field names. These are used to store auxiliary information about the document, such as
|
||||
* its title, url, or an identifier to access a database. The set of stored fields are what is
|
||||
* returned for each hit when searching. This is keyed by document number.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}.
|
||||
* A dictionary containing all of the terms used in all of the
|
||||
* indexed fields of all of the documents. The dictionary also contains the number
|
||||
* of documents which contain the term, and pointers to the term's frequency and
|
||||
* proximity data.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}.
|
||||
* For each term in the dictionary, the numbers of all the
|
||||
* documents that contain that term, and the frequency of the term in that
|
||||
* document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}.
|
||||
* For each term in the dictionary, the positions that the
|
||||
* term occurs in each document. Note that this will not exist if all fields in
|
||||
* all documents omit position data.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}.
|
||||
* For each field in each document, a value is stored
|
||||
* that is multiplied into the score for hits on that field.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}.
|
||||
* For each field in each document, the term vector (sometimes
|
||||
* called document vector) may be stored. A term vector consists of term text and
|
||||
* 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.lucene70.Lucene70DocValuesFormat 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
|
||||
* searches, per-document values are useful for things like scoring factors.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}.
|
||||
* An optional file indicating which documents are live.
|
||||
* </li>
|
||||
* </ul>
|
||||
* <p>Details on each of these are provided in their linked pages.</p>
|
||||
* </div>
|
||||
* <a name="File_Naming"></a>
|
||||
* <h2>File Naming</h2>
|
||||
* <div>
|
||||
* <p>All files belonging to a segment have the same name with varying extensions.
|
||||
* The extensions correspond to the different file formats described below. When
|
||||
* using the Compound File format (default for small segments) these files (except
|
||||
* for the Segment info file, the Lock file, and Deleted documents file) are collapsed
|
||||
* into a single .cfs file (see below for details)</p>
|
||||
* <p>Typically, all segments in an index are stored in a single directory,
|
||||
* although this is not required.</p>
|
||||
* <p>File names are never re-used. That is, when any file is saved
|
||||
* to the Directory it is given a never before used filename. This is achieved
|
||||
* using a simple generations approach. For example, the first segments file is
|
||||
* segments_1, then segments_2, etc. The generation is a sequential long integer
|
||||
* represented in alpha-numeric (base 36) form.</p>
|
||||
* </div>
|
||||
* <a name="file-names"></a>
|
||||
* <h2>Summary of File Extensions</h2>
|
||||
* <div>
|
||||
* <p>The following table summarizes the names and extensions of the files in
|
||||
* Lucene:</p>
|
||||
* <table cellspacing="1" cellpadding="4" summary="lucene filenames by extension">
|
||||
* <tr>
|
||||
* <th>Name</th>
|
||||
* <th>Extension</th>
|
||||
* <th>Brief Description</th>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
|
||||
* <td>segments_N</td>
|
||||
* <td>Stores information about a commit point</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td><a href="#Lock_File">Lock File</a></td>
|
||||
* <td>write.lock</td>
|
||||
* <td>The Write lock prevents multiple IndexWriters from writing to the same
|
||||
* file.</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment Info}</td>
|
||||
* <td>.si</td>
|
||||
* <td>Stores metadata about a segment</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}</td>
|
||||
* <td>.cfs, .cfe</td>
|
||||
* <td>An optional "virtual" file consisting of all the other index files for
|
||||
* systems that frequently run out of file handles.</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}</td>
|
||||
* <td>.fnm</td>
|
||||
* <td>Stores information about the fields</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
|
||||
* <td>.fdx</td>
|
||||
* <td>Contains pointers to field data</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
|
||||
* <td>.fdt</td>
|
||||
* <td>The stored fields for documents</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
|
||||
* <td>.tim</td>
|
||||
* <td>The term dictionary, stores term info</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
|
||||
* <td>.tip</td>
|
||||
* <td>The index into the Term Dictionary</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
|
||||
* <td>.doc</td>
|
||||
* <td>Contains the list of docs which contain each term along with frequency</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
|
||||
* <td>.pos</td>
|
||||
* <td>Stores position information about where a term occurs in the index</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
|
||||
* <td>.pay</td>
|
||||
* <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>.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>.dvd, .dvm</td>
|
||||
* <td>Encodes additional scoring factors or other per-document information.</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
|
||||
* <td>.tvx</td>
|
||||
* <td>Stores offset into the document data file</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Data}</td>
|
||||
* <td>.tvd</td>
|
||||
* <td>Contains term vector data.</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
|
||||
* <td>.liv</td>
|
||||
* <td>Info about what files are live</td>
|
||||
* </tr>
|
||||
* </table>
|
||||
* </div>
|
||||
* <a name="Lock_File"></a>
|
||||
* <h2>Lock File</h2>
|
||||
* The write lock, which is stored in the index directory by default, is named
|
||||
* "write.lock". If the lock directory is different from the index directory then
|
||||
* the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
|
||||
* derived from the full path to the index directory. When this file is present, a
|
||||
* writer is currently modifying the index (adding or removing documents). This
|
||||
* lock file ensures that only one writer is modifying the index at a time.
|
||||
* <a name="History"></a>
|
||||
* <h2>History</h2>
|
||||
* <p>Compatibility notes are provided in this document, describing how file
|
||||
* formats have changed from prior versions:</p>
|
||||
* <ul>
|
||||
* <li>In version 2.1, the file format was changed to allow lock-less commits (ie,
|
||||
* no more commit lock). The change is fully backwards compatible: you can open a
|
||||
* pre-2.1 index for searching or adding/deleting of docs. When the new segments
|
||||
* file is saved (committed), it will be written in the new file format (meaning
|
||||
* no specific "upgrade" process is needed). But note that once a commit has
|
||||
* occurred, pre-2.1 Lucene will not be able to read the index.</li>
|
||||
* <li>In version 2.3, the file format was changed to allow segments to share a
|
||||
* single set of doc store (vectors & stored fields) files. This allows for
|
||||
* faster indexing in certain cases. The change is fully backwards compatible (in
|
||||
* the same way as the lock-less commits change in 2.1).</li>
|
||||
* <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
|
||||
* Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
|
||||
* LUCENE-510</a> for details.</li>
|
||||
* <li>In version 2.9, an optional opaque Map<String,String> CommitUserData
|
||||
* may be passed to IndexWriter's commit methods (and later retrieved), which is
|
||||
* recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
|
||||
* LUCENE-1382</a> for details. Also,
|
||||
* diagnostics were added to each segment written recording details about why it
|
||||
* was written (due to flush, merge; which OS/JRE was used; etc.). See issue
|
||||
* <a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
|
||||
* <li>In version 3.0, compressed fields are no longer written to the index (they
|
||||
* can still be read, but on merge the new segment will write them, uncompressed).
|
||||
* See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
|
||||
* for details.</li>
|
||||
* <li>In version 3.1, segments records the code version that created them. See
|
||||
* <a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
|
||||
* Additionally segments track explicitly whether or not they have term vectors.
|
||||
* See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
|
||||
* for details.</li>
|
||||
* <li>In version 3.2, numeric fields are written as natively to stored fields
|
||||
* file, previously they were stored in text format only.</li>
|
||||
* <li>In version 3.4, fields can omit position data while still indexing term
|
||||
* frequencies.</li>
|
||||
* <li>In version 4.0, the format of the inverted index became extensible via
|
||||
* the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
|
||||
* ({@code DocValues}) was introduced. Normalization factors need no longer be a
|
||||
* single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
|
||||
* Terms need not be unicode strings, they can be any byte sequence. Term offsets
|
||||
* can optionally be indexed into the postings lists. Payloads can be stored in the
|
||||
* term vectors.</li>
|
||||
* <li>In version 4.1, the format of the postings list changed to use either
|
||||
* of FOR compression or variable-byte encoding, depending upon the frequency
|
||||
* of the term. Terms appearing only once were changed to inline directly into
|
||||
* the term dictionary. Stored fields are compressed by default. </li>
|
||||
* <li>In version 4.2, term vectors are compressed by default. DocValues has
|
||||
* a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
|
||||
* on multi-valued fields.</li>
|
||||
* <li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
|
||||
* <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
|
||||
* allow updating NumericDocValues fields.</li>
|
||||
* <li>In version 4.8, checksum footers were added to the end of each index file
|
||||
* for improved data integrity. Specifically, the last 8 bytes of every index file
|
||||
* contain the zlib-crc32 checksum of the file.</li>
|
||||
* <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
|
||||
* that is suitable for faceting/sorting/analytics.
|
||||
* <li>In version 5.4, DocValues have been improved to store more information on disk:
|
||||
* addresses for binary fields and ord indexes for multi-valued fields.
|
||||
* <li>In version 7.0, DocValues have been improved to better support sparse doc values
|
||||
* thanks to an iterator API.
|
||||
* </li>
|
||||
* </ul>
|
||||
* <a name="Limitations"></a>
|
||||
* <h2>Limitations</h2>
|
||||
* <div>
|
||||
* <p>Lucene uses a Java <code>int</code> to refer to
|
||||
* document numbers, and the index file format uses an <code>Int32</code>
|
||||
* on-disk to store document numbers. This is a limitation
|
||||
* of both the index file format and the current implementation. Eventually these
|
||||
* should be replaced with either <code>UInt64</code> values, or
|
||||
* better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
|
||||
* </div>
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene70;
|
|
@ -14,3 +14,4 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene62.Lucene62Codec
|
||||
org.apache.lucene.codecs.lucene70.Lucene70Codec
|
||||
|
|
|
@ -14,3 +14,4 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat
|
||||
org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.codecs.lucene50;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
|
||||
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
|
||||
|
@ -33,7 +33,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
|||
public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return new Lucene62Codec(Mode.BEST_COMPRESSION);
|
||||
return new Lucene70Codec(Mode.BEST_COMPRESSION);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -44,7 +44,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
|
|||
Directory dir = newDirectory();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setCodec(new Lucene62Codec(RandomPicks.randomFrom(random(), Mode.values())));
|
||||
iwc.setCodec(new Lucene70Codec(RandomPicks.randomFrom(random(), Mode.values())));
|
||||
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
|
||||
Document doc = new Document();
|
||||
doc.add(new StoredField("field1", "value1"));
|
||||
|
@ -71,7 +71,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
|
|||
|
||||
public void testInvalidOptions() throws Exception {
|
||||
expectThrows(NullPointerException.class, () -> {
|
||||
new Lucene62Codec(null);
|
||||
new Lucene70Codec(null);
|
||||
});
|
||||
|
||||
expectThrows(NullPointerException.class, () -> {
|
||||
|
|
|
@ -18,14 +18,14 @@ package org.apache.lucene.codecs.lucene53;
|
|||
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
|
||||
import org.apache.lucene.index.BaseNormsFormatTestCase;
|
||||
|
||||
/**
|
||||
* Tests Lucene53NormsFormat
|
||||
*/
|
||||
public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
|
||||
private final Codec codec = new Lucene62Codec();
|
||||
private final Codec codec = new Lucene70Codec();
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
|
|
|
@ -0,0 +1,613 @@
|
|||
/*
|
||||
* 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.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
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.lucene70.Lucene70DocValuesProducer.SparseNumericDocValues;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesProducer.SparseNumericDocValuesRandomAccessWrapper;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.SortedNumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SerialMergeScheduler;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.RAMFile;
|
||||
import org.apache.lucene.store.RAMInputStream;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.LongValues;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Tests Lucene70DocValuesFormat
|
||||
*/
|
||||
public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene70DocValuesFormat());
|
||||
|
||||
@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.
|
||||
|
||||
@Slow
|
||||
public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
|
||||
}
|
||||
}
|
||||
|
||||
@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, 100);
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
public void testTermsEnumFixedWidth() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
public void testSparseDocValuesVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSparseDocValuesVsStoredFields();
|
||||
}
|
||||
}
|
||||
|
||||
private void doTestSparseDocValuesVsStoredFields() throws Exception {
|
||||
final long[] values = new long[TestUtil.nextInt(random(), 1, 500)];
|
||||
for (int i = 0; i < values.length; ++i) {
|
||||
values[i] = random().nextLong();
|
||||
}
|
||||
|
||||
Directory dir = newFSDirectory(createTempDir());
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setMergeScheduler(new SerialMergeScheduler());
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
|
||||
|
||||
// sparse compression is only enabled if less than 1% of docs have a value
|
||||
final int avgGap = 100;
|
||||
|
||||
final int numDocs = atLeast(200);
|
||||
for (int i = random().nextInt(avgGap * 2); i >= 0; --i) {
|
||||
writer.addDocument(new Document());
|
||||
}
|
||||
final int maxNumValuesPerDoc = random().nextBoolean() ? 1 : TestUtil.nextInt(random(), 2, 5);
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
Document doc = new Document();
|
||||
|
||||
// single-valued
|
||||
long docValue = values[random().nextInt(values.length)];
|
||||
doc.add(new NumericDocValuesField("numeric", docValue));
|
||||
doc.add(new SortedDocValuesField("sorted", new BytesRef(Long.toString(docValue))));
|
||||
doc.add(new BinaryDocValuesField("binary", new BytesRef(Long.toString(docValue))));
|
||||
doc.add(new StoredField("value", docValue));
|
||||
|
||||
// multi-valued
|
||||
final int numValues = TestUtil.nextInt(random(), 1, maxNumValuesPerDoc);
|
||||
for (int j = 0; j < numValues; ++j) {
|
||||
docValue = values[random().nextInt(values.length)];
|
||||
doc.add(new SortedNumericDocValuesField("sorted_numeric", docValue));
|
||||
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Long.toString(docValue))));
|
||||
doc.add(new StoredField("values", docValue));
|
||||
}
|
||||
|
||||
writer.addDocument(doc);
|
||||
|
||||
// add a gap
|
||||
for (int j = TestUtil.nextInt(random(), 0, avgGap * 2); j >= 0; --j) {
|
||||
writer.addDocument(new Document());
|
||||
}
|
||||
}
|
||||
|
||||
if (random().nextBoolean()) {
|
||||
writer.forceMerge(1);
|
||||
}
|
||||
|
||||
final IndexReader indexReader = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
for (LeafReaderContext context : indexReader.leaves()) {
|
||||
final LeafReader reader = context.reader();
|
||||
final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric");
|
||||
|
||||
final SortedDocValues sorted = DocValues.getSorted(reader, "sorted");
|
||||
|
||||
final BinaryDocValues binary = DocValues.getBinary(reader, "binary");
|
||||
|
||||
final SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(reader, "sorted_numeric");
|
||||
|
||||
final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set");
|
||||
|
||||
for (int i = 0; i < reader.maxDoc(); ++i) {
|
||||
final Document doc = reader.document(i);
|
||||
final IndexableField valueField = doc.getField("value");
|
||||
final Long value = valueField == null ? null : valueField.numericValue().longValue();
|
||||
|
||||
if (value == null) {
|
||||
assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i);
|
||||
} else {
|
||||
assertEquals(i, numeric.nextDoc());
|
||||
assertEquals(i, binary.nextDoc());
|
||||
assertEquals(i, sorted.nextDoc());
|
||||
assertEquals(value.longValue(), numeric.longValue());
|
||||
assertTrue(sorted.ordValue() >= 0);
|
||||
assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue()));
|
||||
assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue());
|
||||
}
|
||||
|
||||
final IndexableField[] valuesFields = doc.getFields("values");
|
||||
if (valuesFields.length == 0) {
|
||||
assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i);
|
||||
} else {
|
||||
final Set<Long> valueSet = new HashSet<>();
|
||||
for (IndexableField sf : valuesFields) {
|
||||
valueSet.add(sf.numericValue().longValue());
|
||||
}
|
||||
|
||||
assertEquals(i, sortedNumeric.nextDoc());
|
||||
assertEquals(valuesFields.length, sortedNumeric.docValueCount());
|
||||
for (int j = 0; j < sortedNumeric.docValueCount(); ++j) {
|
||||
assertTrue(valueSet.contains(sortedNumeric.nextValue()));
|
||||
}
|
||||
assertEquals(i, sortedSet.nextDoc());
|
||||
int sortedSetCount = 0;
|
||||
while (true) {
|
||||
long ord = sortedSet.nextOrd();
|
||||
if (ord == SortedSetDocValues.NO_MORE_ORDS) {
|
||||
break;
|
||||
}
|
||||
assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString())));
|
||||
sortedSetCount++;
|
||||
}
|
||||
assertEquals(valueSet.size(), sortedSetCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
indexReader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// 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 = TestUtil.getPostingsFormatWithOrds(random());
|
||||
final DocValuesFormat dv = new Lucene70DocValuesFormat();
|
||||
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) {
|
||||
SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
|
||||
assertEquals(terms.size(), ssdv.getValueCount());
|
||||
TermsEnum expected = terms.iterator();
|
||||
TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
|
||||
assertEquals(terms.size(), expected, actual);
|
||||
|
||||
doTestSortedSetEnumAdvanceIndependently(ssdv);
|
||||
}
|
||||
}
|
||||
ir.close();
|
||||
|
||||
writer.forceMerge(1);
|
||||
|
||||
// now compare again after the merge
|
||||
ir = writer.getReader();
|
||||
LeafReader ar = getOnlyLeafReader(ir);
|
||||
Terms terms = ar.terms("indexed");
|
||||
if (terms != null) {
|
||||
assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
|
||||
TermsEnum expected = terms.iterator();
|
||||
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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testSparseLongValues() throws IOException {
|
||||
final int iters = atLeast(5);
|
||||
for (int iter = 0; iter < iters; ++iter) {
|
||||
final int numDocs = TestUtil.nextInt(random(), 0, 100);
|
||||
final int[] docIds = new int[numDocs];
|
||||
final long[] values = new long[numDocs];
|
||||
final int maxDoc;
|
||||
if (numDocs == 0) {
|
||||
maxDoc = 1 + random().nextInt(10);
|
||||
} else {
|
||||
docIds[0] = random().nextInt(10);
|
||||
for (int i = 1; i < docIds.length; ++i) {
|
||||
docIds[i] = docIds[i - 1] + 1 + random().nextInt(100);
|
||||
}
|
||||
maxDoc = docIds[numDocs - 1] + 1 + random().nextInt(10);
|
||||
}
|
||||
for (int i = 0; i < values.length; ++i) {
|
||||
values[i] = random().nextLong();
|
||||
}
|
||||
final long missingValue = random().nextLong();
|
||||
final LongValues docIdsValues = new LongValues() {
|
||||
@Override
|
||||
public long get(long index) {
|
||||
return docIds[Math.toIntExact(index)];
|
||||
}
|
||||
};
|
||||
final LongValues valuesValues = new LongValues() {
|
||||
@Override
|
||||
public long get(long index) {
|
||||
return values[Math.toIntExact(index)];
|
||||
}
|
||||
};
|
||||
final SparseNumericDocValues sparseValues = new SparseNumericDocValues(numDocs, docIdsValues, valuesValues);
|
||||
|
||||
// sequential access
|
||||
assertEquals(-1, sparseValues.docID());
|
||||
for (int i = 0; i < docIds.length; ++i) {
|
||||
assertEquals(docIds[i], sparseValues.nextDoc());
|
||||
}
|
||||
assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.nextDoc());
|
||||
|
||||
// advance
|
||||
for (int i = 0; i < 2000; ++i) {
|
||||
final int target = TestUtil.nextInt(random(), 0, (int) maxDoc);
|
||||
int index = Arrays.binarySearch(docIds, target);
|
||||
if (index < 0) {
|
||||
index = -1 - index;
|
||||
}
|
||||
sparseValues.reset();
|
||||
if (index > 0) {
|
||||
assertEquals(docIds[index - 1], sparseValues.advance(Math.toIntExact(docIds[index - 1])));
|
||||
}
|
||||
if (index == docIds.length) {
|
||||
assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.advance(target));
|
||||
} else {
|
||||
assertEquals(docIds[index], sparseValues.advance(target));
|
||||
}
|
||||
}
|
||||
|
||||
final SparseNumericDocValuesRandomAccessWrapper raWrapper = new SparseNumericDocValuesRandomAccessWrapper(sparseValues, missingValue);
|
||||
|
||||
// random-access
|
||||
for (int i = 0; i < 2000; ++i) {
|
||||
final int docId = TestUtil.nextInt(random(), 0, maxDoc - 1);
|
||||
final int idx = Arrays.binarySearch(docIds, docId);
|
||||
final long value = raWrapper.get(docId);
|
||||
if (idx >= 0) {
|
||||
assertEquals(values[idx], value);
|
||||
} else {
|
||||
assertEquals(missingValue, value);
|
||||
}
|
||||
}
|
||||
|
||||
// sequential access
|
||||
for (int docId = 0; docId < maxDoc; docId += random().nextInt(3)) {
|
||||
final int idx = Arrays.binarySearch(docIds, docId);
|
||||
final long value = raWrapper.get(docId);
|
||||
if (idx >= 0) {
|
||||
assertEquals(values[idx], value);
|
||||
} else {
|
||||
assertEquals(missingValue, value);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
public void testSortedSetAroundBlockSize() throws IOException {
|
||||
final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
|
||||
for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
|
||||
final Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
|
||||
RAMFile buffer = new RAMFile();
|
||||
RAMOutputStream out = new RAMOutputStream(buffer, false);
|
||||
Document doc = new Document();
|
||||
SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef());
|
||||
doc.add(field1);
|
||||
SortedSetDocValuesField field2 = new SortedSetDocValuesField("sset", new BytesRef());
|
||||
doc.add(field2);
|
||||
for (int i = 0; i < maxDoc; ++i) {
|
||||
BytesRef s1 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
|
||||
BytesRef s2 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
|
||||
field1.setBytesValue(s1);
|
||||
field2.setBytesValue(s2);
|
||||
w.addDocument(doc);
|
||||
Set<BytesRef> set = new TreeSet<>(Arrays.asList(s1, s2));
|
||||
out.writeVInt(set.size());
|
||||
for (BytesRef ref : set) {
|
||||
out.writeVInt(ref.length);
|
||||
out.writeBytes(ref.bytes, ref.offset, ref.length);
|
||||
}
|
||||
}
|
||||
out.close();
|
||||
w.forceMerge(1);
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
w.close();
|
||||
LeafReader sr = getOnlyLeafReader(r);
|
||||
assertEquals(maxDoc, sr.maxDoc());
|
||||
SortedSetDocValues values = sr.getSortedSetDocValues("sset");
|
||||
assertNotNull(values);
|
||||
RAMInputStream in = new RAMInputStream("", buffer);
|
||||
BytesRefBuilder b = new BytesRefBuilder();
|
||||
for (int i = 0; i < maxDoc; ++i) {
|
||||
assertEquals(i, values.nextDoc());
|
||||
final int numValues = in.readVInt();
|
||||
|
||||
for (int j = 0; j < numValues; ++j) {
|
||||
b.setLength(in.readVInt());
|
||||
b.grow(b.length());
|
||||
in.readBytes(b.bytes(), 0, b.length());
|
||||
assertEquals(b.get(), values.lookupOrd(values.nextOrd()));
|
||||
}
|
||||
|
||||
assertEquals(SortedSetDocValues.NO_MORE_ORDS, values.nextOrd());
|
||||
}
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
public void testSortedNumericAroundBlockSize() throws IOException {
|
||||
final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
|
||||
for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
|
||||
final Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
|
||||
RAMFile buffer = new RAMFile();
|
||||
RAMOutputStream out = new RAMOutputStream(buffer, false);
|
||||
Document doc = new Document();
|
||||
SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L);
|
||||
doc.add(field1);
|
||||
SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("snum", 0L);
|
||||
doc.add(field2);
|
||||
for (int i = 0; i < maxDoc; ++i) {
|
||||
long s1 = random().nextInt(100);
|
||||
long s2 = random().nextInt(100);
|
||||
field1.setLongValue(s1);
|
||||
field2.setLongValue(s2);
|
||||
w.addDocument(doc);
|
||||
out.writeVLong(Math.min(s1, s2));
|
||||
out.writeVLong(Math.max(s1, s2));
|
||||
}
|
||||
out.close();
|
||||
w.forceMerge(1);
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
w.close();
|
||||
LeafReader sr = getOnlyLeafReader(r);
|
||||
assertEquals(maxDoc, sr.maxDoc());
|
||||
SortedNumericDocValues values = sr.getSortedNumericDocValues("snum");
|
||||
assertNotNull(values);
|
||||
RAMInputStream in = new RAMInputStream("", buffer);
|
||||
for (int i = 0; i < maxDoc; ++i) {
|
||||
assertEquals(i, values.nextDoc());
|
||||
assertEquals(2, values.docValueCount());
|
||||
assertEquals(in.readVLong(), values.nextValue());
|
||||
assertEquals(in.readVLong(), values.nextValue());
|
||||
}
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -135,6 +135,6 @@ public class Test2BPoints extends LuceneTestCase {
|
|||
}
|
||||
|
||||
private static Codec getCodec() {
|
||||
return Codec.forName("Lucene62");
|
||||
return Codec.forName("Lucene70");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -393,11 +393,11 @@ public class TestPointValues extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
// Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with SimpleText
|
||||
// Write point values, one segment with Lucene70, another with SimpleText, then forceMerge with SimpleText
|
||||
public void testDifferentCodecs1() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(Codec.forName("Lucene62"));
|
||||
iwc.setCodec(Codec.forName("Lucene70"));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new IntPoint("int", 1));
|
||||
|
@ -416,7 +416,7 @@ public class TestPointValues extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
// Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with Lucene60
|
||||
// Write point values, one segment with Lucene70, another with SimpleText, then forceMerge with Lucene70
|
||||
public void testDifferentCodecs2() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -428,7 +428,7 @@ public class TestPointValues extends LuceneTestCase {
|
|||
w.close();
|
||||
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(Codec.forName("Lucene62"));
|
||||
iwc.setCodec(Codec.forName("Lucene70"));
|
||||
w = new IndexWriter(dir, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new IntPoint("int", 1));
|
||||
|
|
|
@ -96,7 +96,7 @@ public class TestBoolean2 extends LuceneTestCase {
|
|||
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
// randomized codecs are sometimes too costly for this test:
|
||||
iwc.setCodec(Codec.forName("Lucene62"));
|
||||
iwc.setCodec(Codec.forName("Lucene70"));
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter writer= new RandomIndexWriter(random(), directory, iwc);
|
||||
// we'll make a ton of docs, disable store/norms/vectors
|
||||
|
@ -141,7 +141,7 @@ public class TestBoolean2 extends LuceneTestCase {
|
|||
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
// we need docID order to be preserved:
|
||||
// randomized codecs are sometimes too costly for this test:
|
||||
iwc.setCodec(Codec.forName("Lucene62"));
|
||||
iwc.setCodec(Codec.forName("Lucene70"));
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
try (IndexWriter w = new IndexWriter(singleSegmentDirectory, iwc)) {
|
||||
w.forceMerge(1, true);
|
||||
|
@ -167,7 +167,7 @@ public class TestBoolean2 extends LuceneTestCase {
|
|||
|
||||
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
// randomized codecs are sometimes too costly for this test:
|
||||
iwc.setCodec(Codec.forName("Lucene62"));
|
||||
iwc.setCodec(Codec.forName("Lucene70"));
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
|
||||
w.addIndexes(copy);
|
||||
copy.close();
|
||||
|
@ -179,7 +179,7 @@ public class TestBoolean2 extends LuceneTestCase {
|
|||
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000));
|
||||
// randomized codecs are sometimes too costly for this test:
|
||||
iwc.setCodec(Codec.forName("Lucene62"));
|
||||
iwc.setCodec(Codec.forName("Lucene70"));
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
|
||||
|
||||
doc = new Document();
|
||||
|
|
|
@ -1157,14 +1157,14 @@ public class TestPointQueries extends LuceneTestCase {
|
|||
}
|
||||
|
||||
private static Codec getCodec() {
|
||||
if (Codec.getDefault().getName().equals("Lucene62")) {
|
||||
if (Codec.getDefault().getName().equals("Lucene70")) {
|
||||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
|
||||
double maxMBSortInHeap = 5.0 + (3*random().nextDouble());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
}
|
||||
|
||||
return new FilterCodec("Lucene62", Codec.getDefault()) {
|
||||
return new FilterCodec("Lucene70", Codec.getDefault()) {
|
||||
@Override
|
||||
public PointsFormat pointsFormat() {
|
||||
return new PointsFormat() {
|
||||
|
|
|
@ -247,7 +247,7 @@ public class TestNearest extends LuceneTestCase {
|
|||
|
||||
private IndexWriterConfig getIndexWriterConfig() {
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setCodec(Codec.forName("Lucene62"));
|
||||
iwc.setCodec(Codec.forName("Lucene70"));
|
||||
return iwc;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -85,14 +85,14 @@ import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
|||
public class TestGeo3DPoint extends LuceneTestCase {
|
||||
|
||||
private static Codec getCodec() {
|
||||
if (Codec.getDefault().getName().equals("Lucene62")) {
|
||||
if (Codec.getDefault().getName().equals("Lucene70")) {
|
||||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
|
||||
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
}
|
||||
|
||||
return new FilterCodec("Lucene62", Codec.getDefault()) {
|
||||
return new FilterCodec("Lucene70", Codec.getDefault()) {
|
||||
@Override
|
||||
public PointsFormat pointsFormat() {
|
||||
return new PointsFormat() {
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -646,7 +646,7 @@ public class TestSuggestField extends LuceneTestCase {
|
|||
static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, final Set<String> suggestFields) {
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
Codec filterCodec = new Lucene62Codec() {
|
||||
Codec filterCodec = new Lucene70Codec() {
|
||||
PostingsFormat postingsFormat = new Completion50PostingsFormat();
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1239,7 +1239,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
|
|||
// Else seeds may not reproduce:
|
||||
iwc.setMergeScheduler(new SerialMergeScheduler());
|
||||
int pointsInLeaf = 2 + random().nextInt(4);
|
||||
iwc.setCodec(new FilterCodec("Lucene62", TestUtil.getDefaultCodec()) {
|
||||
iwc.setCodec(new FilterCodec("Lucene70", TestUtil.getDefaultCodec()) {
|
||||
@Override
|
||||
public PointsFormat pointsFormat() {
|
||||
return new PointsFormat() {
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
|
|||
import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
|
||||
import org.apache.lucene.codecs.compressing.CompressingCodec;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
|
||||
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.index.RandomCodec;
|
||||
|
@ -181,8 +181,8 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
codec = new AssertingCodec();
|
||||
} else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
|
||||
codec = CompressingCodec.randomInstance(random);
|
||||
} else if ("Lucene62".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene62"))) {
|
||||
codec = new Lucene62Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
|
||||
} else if ("Lucene70".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene70"))) {
|
||||
codec = new Lucene70Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
|
||||
} else if (!"random".equals(TEST_CODEC)) {
|
||||
codec = Codec.forName(TEST_CODEC);
|
||||
} else if ("random".equals(TEST_POSTINGSFORMAT)) {
|
||||
|
|
|
@ -53,8 +53,8 @@ import org.apache.lucene.codecs.asserting.AssertingCodec;
|
|||
import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
|
||||
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
|
@ -913,7 +913,7 @@ public final class TestUtil {
|
|||
* This may be different than {@link Codec#getDefault()} because that is randomized.
|
||||
*/
|
||||
public static Codec getDefaultCodec() {
|
||||
return new Lucene62Codec();
|
||||
return new Lucene70Codec();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -946,7 +946,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 Lucene54DocValuesFormat();
|
||||
return new Lucene70DocValuesFormat();
|
||||
}
|
||||
|
||||
// TODO: generalize all 'test-checks-for-crazy-codecs' to
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
|
||||
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
|
@ -91,7 +91,7 @@ public class SchemaCodecFactory extends CodecFactory implements SolrCoreAware {
|
|||
compressionMode = SOLR_DEFAULT_COMPRESSION_MODE;
|
||||
log.debug("Using default compressionMode: " + compressionMode);
|
||||
}
|
||||
codec = new Lucene62Codec(compressionMode) {
|
||||
codec = new Lucene70Codec(compressionMode) {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
<fieldType name="string_direct" class="solr.StrField" postingsFormat="Direct"/>
|
||||
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene50"/>
|
||||
|
||||
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene54"/>
|
||||
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene70"/>
|
||||
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory"/>
|
||||
|
||||
<fieldType name="string" class="solr.StrField"/>
|
||||
|
|
Loading…
Reference in New Issue