mirror of https://github.com/apache/lucene.git
LUCENE-5882: Add 4.10 docvaluesformat
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1617975 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b0f63241dc
commit
99c23632fa
|
@ -217,6 +217,9 @@ Optimizations
|
|||
* LUCENE-5884: Optimize FST.ramBytesUsed. (Adrien Grand, Robert Muir,
|
||||
Mike McCandless)
|
||||
|
||||
* LUCENE-5882: Add Lucene410DocValuesFormat, with faster term lookups
|
||||
for SORTED/SORTED_SET fields. (Robert Muir)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-5796: Fixes the Scorer.getChildren() method for two combinations
|
||||
|
|
|
@ -21,7 +21,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.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.index.ConcurrentMergeScheduler;
|
||||
import org.apache.lucene.index.IndexCommit;
|
||||
import org.apache.lucene.index.IndexDeletionPolicy;
|
||||
|
@ -140,7 +140,7 @@ public class CreateIndexTask extends PerfTask {
|
|||
if (defaultCodec == null && postingsFormat != null) {
|
||||
try {
|
||||
final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
|
||||
iwConf.setCodec(new Lucene49Codec(){
|
||||
iwConf.setCodec(new Lucene410Codec(){
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return postingsFormatChosen;
|
||||
|
|
|
@ -288,7 +288,7 @@
|
|||
<classpath>
|
||||
<path refid="groovy.classpath"/>
|
||||
</classpath><![CDATA[
|
||||
// private static Codec defaultCodec = Codec . forName ( "Lucene49" ) ;
|
||||
// private static Codec defaultCodec = Codec . forName ( "Lucene410" ) ;
|
||||
defaultCodecMatcher = self.getToken() =~ /defaultCodec\s*=\s*Codec\s*\.\s*forName\s*\(\s*"([^"]+)"\s*\)\s*;/
|
||||
self.setToken(defaultCodecMatcher[0][1].toLowerCase());
|
||||
]]>
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
|
@ -105,15 +106,7 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
// As long as codec sorts terms in unicode codepoint
|
||||
// order, we can safely strip off the non-distinguishing
|
||||
// suffix to save RAM in the loaded terms index.
|
||||
final int idxTermOffset = indexedTerm.offset;
|
||||
final int priorTermOffset = priorTerm.offset;
|
||||
final int limit = Math.min(priorTerm.length, indexedTerm.length);
|
||||
for(int byteIdx=0;byteIdx<limit;byteIdx++) {
|
||||
if (priorTerm.bytes[priorTermOffset+byteIdx] != indexedTerm.bytes[idxTermOffset+byteIdx]) {
|
||||
return byteIdx+1;
|
||||
}
|
||||
}
|
||||
return Math.min(1+priorTerm.length, indexedTerm.length);
|
||||
return StringHelper.sortKeyLength(priorTerm, indexedTerm);
|
||||
}
|
||||
|
||||
private class SimpleFieldWriter extends FieldWriter {
|
||||
|
|
|
@ -119,7 +119,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
loader.reload(classloader);
|
||||
}
|
||||
|
||||
private static Codec defaultCodec = Codec.forName("Lucene49");
|
||||
private static Codec defaultCodec = Codec.forName("Lucene410");
|
||||
|
||||
/** expert: returns the default codec used for newly created
|
||||
* {@link IndexWriterConfig}s.
|
||||
|
|
|
@ -21,13 +21,13 @@ package org.apache.lucene.codecs;
|
|||
* A codec that forwards all its method calls to another codec.
|
||||
* <p>
|
||||
* Extend this class when you need to reuse the functionality of an existing
|
||||
* codec. For example, if you want to build a codec that redefines Lucene49's
|
||||
* codec. For example, if you want to build a codec that redefines Lucene410's
|
||||
* {@link LiveDocsFormat}:
|
||||
* <pre class="prettyprint">
|
||||
* public final class CustomCodec extends FilterCodec {
|
||||
*
|
||||
* public CustomCodec() {
|
||||
* super("CustomCodec", new Lucene49Codec());
|
||||
* super("CustomCodec", new Lucene410Codec());
|
||||
* }
|
||||
*
|
||||
* public LiveDocsFormat liveDocsFormat() {
|
||||
|
|
|
@ -0,0 +1,140 @@
|
|||
package org.apache.lucene.codecs.lucene410;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
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.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene46.Lucene46FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene46.Lucene46SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 4.10 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.lucene410 package documentation for file format details.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
// NOTE: if we make largish changes in a minor release, easier to just make Lucene411Codec or whatever
|
||||
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
|
||||
// (it writes a minor version, etc).
|
||||
public class Lucene410Codec extends Codec {
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
|
||||
private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
|
||||
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return Lucene410Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return Lucene410Codec.this.getDocValuesFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene410Codec() {
|
||||
super("Lucene410");
|
||||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene41"
|
||||
*/
|
||||
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 "Lucene410"
|
||||
*/
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return defaultDVFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene410");
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene49NormsFormat();
|
||||
|
||||
@Override
|
||||
public final NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,559 @@
|
|||
package org.apache.lucene.codecs.lucene410;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.Closeable; // javadocs
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.MathUtil;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.packed.DirectWriter;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/** writer for {@link Lucene410DocValuesFormat} */
|
||||
class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
||||
|
||||
static final int BLOCK_SIZE = 16384;
|
||||
|
||||
// address terms in blocks of 16 terms
|
||||
static final int INTERVAL_SHIFT = 4;
|
||||
static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
|
||||
static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
|
||||
|
||||
// build reverse index from every 1024th term
|
||||
static final int REVERSE_INTERVAL_SHIFT = 10;
|
||||
static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
|
||||
static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
|
||||
|
||||
// for conversion from reverse index to block
|
||||
static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
|
||||
static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
|
||||
static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
|
||||
|
||||
/** Compressed using packed blocks of ints. */
|
||||
public static final int DELTA_COMPRESSED = 0;
|
||||
/** Compressed by computing the GCD. */
|
||||
public static final int GCD_COMPRESSED = 1;
|
||||
/** Compressed by giving IDs to unique values. */
|
||||
public static final int TABLE_COMPRESSED = 2;
|
||||
/** Compressed with monotonically increasing values */
|
||||
public static final int MONOTONIC_COMPRESSED = 3;
|
||||
|
||||
/** Uncompressed binary, written directly (fixed length). */
|
||||
public static final int BINARY_FIXED_UNCOMPRESSED = 0;
|
||||
/** Uncompressed binary, written directly (variable length). */
|
||||
public static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
|
||||
/** Compressed binary with shared prefixes */
|
||||
public static final int BINARY_PREFIX_COMPRESSED = 2;
|
||||
|
||||
/** Standard storage for sorted set values with 1 level of indirection:
|
||||
* docId -> address -> ord. */
|
||||
public static final int SORTED_WITH_ADDRESSES = 0;
|
||||
/** Single-valued sorted set values, encoded as sorted values, so no level
|
||||
* of indirection: docId -> ord. */
|
||||
public static final int SORTED_SINGLE_VALUED = 1;
|
||||
|
||||
IndexOutput data, meta;
|
||||
final int maxDoc;
|
||||
|
||||
/** expert: Creates a new writer */
|
||||
public Lucene410DocValuesConsumer(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.writeHeader(data, dataCodec, Lucene410DocValuesFormat.VERSION_CURRENT);
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
meta = state.directory.createOutput(metaName, state.context);
|
||||
CodecUtil.writeHeader(meta, metaCodec, Lucene410DocValuesFormat.VERSION_CURRENT);
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
addNumericField(field, values, true);
|
||||
}
|
||||
|
||||
void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
|
||||
long count = 0;
|
||||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
long gcd = 0;
|
||||
boolean missing = false;
|
||||
// TODO: more efficient?
|
||||
HashSet<Long> uniqueValues = null;
|
||||
if (optimizeStorage) {
|
||||
uniqueValues = new HashSet<>();
|
||||
|
||||
for (Number nv : values) {
|
||||
final long v;
|
||||
if (nv == null) {
|
||||
v = 0;
|
||||
missing = true;
|
||||
} else {
|
||||
v = nv.longValue();
|
||||
}
|
||||
|
||||
if (gcd != 1) {
|
||||
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
|
||||
// in that case v - minValue might overflow and make the GCD computation return
|
||||
// wrong results. Since these extreme values are unlikely, we just discard
|
||||
// GCD computation for them
|
||||
gcd = 1;
|
||||
} else if (count != 0) { // minValue needs to be set first
|
||||
gcd = MathUtil.gcd(gcd, v - minValue);
|
||||
}
|
||||
}
|
||||
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
||||
if (uniqueValues != null) {
|
||||
if (uniqueValues.add(v)) {
|
||||
if (uniqueValues.size() > 256) {
|
||||
uniqueValues = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++count;
|
||||
}
|
||||
} else {
|
||||
for (Number nv : values) {
|
||||
long v = nv.longValue();
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
++count;
|
||||
}
|
||||
}
|
||||
|
||||
final long delta = maxValue - minValue;
|
||||
final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
|
||||
final int tableBitsRequired = uniqueValues == null
|
||||
? Integer.MAX_VALUE
|
||||
: DirectWriter.bitsRequired(uniqueValues.size() - 1);
|
||||
|
||||
final int format;
|
||||
if (uniqueValues != null && 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(Lucene410DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(format);
|
||||
if (missing) {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
writeMissingBitset(values);
|
||||
} else {
|
||||
meta.writeLong(-1L);
|
||||
}
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVLong(count);
|
||||
|
||||
switch (format) {
|
||||
case GCD_COMPRESSED:
|
||||
meta.writeLong(minValue);
|
||||
meta.writeLong(gcd);
|
||||
final long maxDelta = (maxValue - minValue) / gcd;
|
||||
final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
|
||||
meta.writeVInt(bits);
|
||||
final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
|
||||
for (Number nv : values) {
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
quotientWriter.add((value - minValue) / gcd);
|
||||
}
|
||||
quotientWriter.finish();
|
||||
break;
|
||||
case DELTA_COMPRESSED:
|
||||
final long minDelta = delta < 0 ? 0 : minValue;
|
||||
meta.writeLong(minDelta);
|
||||
meta.writeVInt(deltaBitsRequired);
|
||||
final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
|
||||
for (Number nv : values) {
|
||||
long v = nv == null ? 0 : nv.longValue();
|
||||
writer.add(v - minDelta);
|
||||
}
|
||||
writer.finish();
|
||||
break;
|
||||
case TABLE_COMPRESSED:
|
||||
final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
|
||||
Arrays.sort(decode);
|
||||
final HashMap<Long,Integer> encode = new HashMap<>();
|
||||
meta.writeVInt(decode.length);
|
||||
for (int i = 0; i < decode.length; i++) {
|
||||
meta.writeLong(decode[i]);
|
||||
encode.put(decode[i], i);
|
||||
}
|
||||
meta.writeVInt(tableBitsRequired);
|
||||
final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
|
||||
for (Number nv : values) {
|
||||
ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
ordsWriter.finish();
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
meta.writeLong(data.getFilePointer());
|
||||
}
|
||||
|
||||
// TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
|
||||
// but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
|
||||
void writeMissingBitset(Iterable<?> values) throws IOException {
|
||||
byte bits = 0;
|
||||
int count = 0;
|
||||
for (Object v : values) {
|
||||
if (count == 8) {
|
||||
data.writeByte(bits);
|
||||
count = 0;
|
||||
bits = 0;
|
||||
}
|
||||
if (v != null) {
|
||||
bits |= 1 << (count & 7);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
if (count > 0) {
|
||||
data.writeByte(bits);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
// write the byte[] data
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene410DocValuesFormat.BINARY);
|
||||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
final long startFP = data.getFilePointer();
|
||||
long count = 0;
|
||||
boolean missing = false;
|
||||
for(BytesRef v : values) {
|
||||
final int length;
|
||||
if (v == null) {
|
||||
length = 0;
|
||||
missing = true;
|
||||
} 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 (missing) {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
writeMissingBitset(values);
|
||||
} else {
|
||||
meta.writeLong(-1L);
|
||||
}
|
||||
meta.writeVInt(minLength);
|
||||
meta.writeVInt(maxLength);
|
||||
meta.writeVLong(count);
|
||||
meta.writeLong(startFP);
|
||||
|
||||
// if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
|
||||
// otherwise, we need to record the length fields...
|
||||
if (minLength != maxLength) {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeVInt(BLOCK_SIZE);
|
||||
|
||||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
writer.add(addr);
|
||||
for (BytesRef v : values) {
|
||||
if (v != null) {
|
||||
addr += v.length;
|
||||
}
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
/** expert: writes a value dictionary for a sorted/sortedset field */
|
||||
private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
|
||||
// first check if its a "fixed-length" terms dict
|
||||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
long numValues = 0;
|
||||
for (BytesRef v : values) {
|
||||
minLength = Math.min(minLength, v.length);
|
||||
maxLength = Math.max(maxLength, v.length);
|
||||
numValues++;
|
||||
}
|
||||
if (minLength == maxLength) {
|
||||
// no index needed: direct addressing by mult
|
||||
addBinaryField(field, values);
|
||||
} else if (numValues < REVERSE_INTERVAL_COUNT) {
|
||||
// low cardinality: waste a few KB of ram, but can't really use fancy index etc
|
||||
addBinaryField(field, values);
|
||||
} else {
|
||||
assert numValues > 0; // we don't have to handle the empty case
|
||||
// header
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene410DocValuesFormat.BINARY);
|
||||
meta.writeVInt(BINARY_PREFIX_COMPRESSED);
|
||||
meta.writeLong(-1L);
|
||||
// now write the bytes: sharing prefixes within a block
|
||||
final long startFP = data.getFilePointer();
|
||||
// currently, we have to store the delta from expected for every 1/nth term
|
||||
// we could avoid this, but its not much and less overall RAM than the previous approach!
|
||||
RAMOutputStream addressBuffer = new RAMOutputStream();
|
||||
MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, BLOCK_SIZE);
|
||||
// buffers up 16 terms
|
||||
RAMOutputStream bytesBuffer = new RAMOutputStream();
|
||||
// buffers up block header
|
||||
RAMOutputStream headerBuffer = new RAMOutputStream();
|
||||
BytesRefBuilder lastTerm = new BytesRefBuilder();
|
||||
lastTerm.grow(maxLength);
|
||||
long count = 0;
|
||||
int suffixDeltas[] = new int[INTERVAL_COUNT];
|
||||
for (BytesRef v : values) {
|
||||
int termPosition = (int) (count & INTERVAL_MASK);
|
||||
if (termPosition == 0) {
|
||||
termAddresses.add(data.getFilePointer() - startFP);
|
||||
// abs-encode first term
|
||||
headerBuffer.writeVInt(v.length);
|
||||
headerBuffer.writeBytes(v.bytes, v.offset, v.length);
|
||||
lastTerm.copyBytes(v);
|
||||
} else {
|
||||
// prefix-code: we only share at most 255 characters, to encode the length as a single
|
||||
// byte and have random access. Larger terms just get less compression.
|
||||
int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
|
||||
bytesBuffer.writeByte((byte) sharedPrefix);
|
||||
bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
|
||||
// we can encode one smaller, because terms are unique.
|
||||
suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
|
||||
}
|
||||
|
||||
count++;
|
||||
// flush block
|
||||
if ((count & INTERVAL_MASK) == 0) {
|
||||
flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
|
||||
}
|
||||
}
|
||||
// flush trailing crap
|
||||
int leftover = (int) (count & INTERVAL_MASK);
|
||||
if (leftover > 0) {
|
||||
Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
|
||||
flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
|
||||
}
|
||||
final long indexStartFP = data.getFilePointer();
|
||||
// write addresses of indexed terms
|
||||
termAddresses.finish();
|
||||
addressBuffer.writeTo(data);
|
||||
addressBuffer = null;
|
||||
termAddresses = null;
|
||||
meta.writeVInt(minLength);
|
||||
meta.writeVInt(maxLength);
|
||||
meta.writeVLong(count);
|
||||
meta.writeLong(startFP);
|
||||
meta.writeLong(indexStartFP);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeVInt(BLOCK_SIZE);
|
||||
addReverseTermIndex(field, values, maxLength);
|
||||
}
|
||||
}
|
||||
|
||||
// writes term dictionary "block"
|
||||
// first term is absolute encoded as vint length + bytes.
|
||||
// lengths of subsequent N terms are encoded as either N bytes or N shorts.
|
||||
// in the double-byte case, the first byte is indicated with -1.
|
||||
// subsequent terms are encoded as byte suffixLength + bytes.
|
||||
private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
|
||||
boolean twoByte = false;
|
||||
for (int i = 1; i < suffixDeltas.length; i++) {
|
||||
if (suffixDeltas[i] > 254) {
|
||||
twoByte = true;
|
||||
}
|
||||
}
|
||||
if (twoByte) {
|
||||
headerBuffer.writeByte((byte)255);
|
||||
for (int i = 1; i < suffixDeltas.length; i++) {
|
||||
headerBuffer.writeShort((short) suffixDeltas[i]);
|
||||
}
|
||||
} else {
|
||||
for (int i = 1; i < suffixDeltas.length; i++) {
|
||||
headerBuffer.writeByte((byte) suffixDeltas[i]);
|
||||
}
|
||||
}
|
||||
headerBuffer.writeTo(data);
|
||||
headerBuffer.reset();
|
||||
bytesBuffer.writeTo(data);
|
||||
bytesBuffer.reset();
|
||||
}
|
||||
|
||||
// writes reverse term index: used for binary searching a term into a range of 64 blocks
|
||||
// for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
|
||||
// terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
|
||||
private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
|
||||
long count = 0;
|
||||
BytesRefBuilder priorTerm = new BytesRefBuilder();
|
||||
priorTerm.grow(maxLength);
|
||||
BytesRef indexTerm = new BytesRef();
|
||||
long startFP = data.getFilePointer();
|
||||
PagedBytes pagedBytes = new PagedBytes(15);
|
||||
MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
|
||||
for (BytesRef b : values) {
|
||||
int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
|
||||
if (termPosition == 0) {
|
||||
int len = StringHelper.sortKeyLength(priorTerm.get(), b);
|
||||
indexTerm.bytes = b.bytes;
|
||||
indexTerm.offset = b.offset;
|
||||
indexTerm.length = len;
|
||||
addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
|
||||
} else if (termPosition == REVERSE_INTERVAL_MASK) {
|
||||
priorTerm.copyBytes(b);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
addresses.finish();
|
||||
long numBytes = pagedBytes.getPointer();
|
||||
pagedBytes.freeze(true);
|
||||
PagedBytesDataInput in = pagedBytes.getDataInput();
|
||||
meta.writeLong(startFP);
|
||||
data.writeVLong(numBytes);
|
||||
data.copyBytes(in, numBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene410DocValuesFormat.SORTED);
|
||||
addTermsDict(field, values);
|
||||
addNumericField(field, docToOrd, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene410DocValuesFormat.SORTED_NUMERIC);
|
||||
if (isSingleValued(docToValueCount)) {
|
||||
meta.writeVInt(SORTED_SINGLE_VALUED);
|
||||
// The field is single-valued, we can encode it as NUMERIC
|
||||
addNumericField(field, singletonView(docToValueCount, values, null));
|
||||
} else {
|
||||
meta.writeVInt(SORTED_WITH_ADDRESSES);
|
||||
// write the stream of values as a numeric field
|
||||
addNumericField(field, values, true);
|
||||
// write the doc -> ord count as a absolute index to the stream
|
||||
addAddresses(field, docToValueCount);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene410DocValuesFormat.SORTED_SET);
|
||||
|
||||
if (isSingleValued(docToOrdCount)) {
|
||||
meta.writeVInt(SORTED_SINGLE_VALUED);
|
||||
// The field is single-valued, we can encode it as SORTED
|
||||
addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
|
||||
} else {
|
||||
meta.writeVInt(SORTED_WITH_ADDRESSES);
|
||||
|
||||
// write the ord -> byte[] as a binary field
|
||||
addTermsDict(field, values);
|
||||
|
||||
// write the stream of ords as a numeric field
|
||||
// NOTE: we could return an iterator that delta-encodes these within a doc
|
||||
addNumericField(field, ords, false);
|
||||
|
||||
// write the doc -> ord count as a absolute index to the stream
|
||||
addAddresses(field, docToOrdCount);
|
||||
}
|
||||
}
|
||||
|
||||
// writes addressing information as MONOTONIC_COMPRESSED integer
|
||||
private void addAddresses(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene410DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(MONOTONIC_COMPRESSED);
|
||||
meta.writeLong(-1L);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVLong(maxDoc);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeVInt(BLOCK_SIZE);
|
||||
|
||||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
writer.add(addr);
|
||||
for (Number v : values) {
|
||||
addr += v.longValue();
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
meta.writeLong(data.getFilePointer());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
if (meta != null) {
|
||||
meta.writeVInt(-1); // write EOF marker
|
||||
CodecUtil.writeFooter(meta); // write checksum
|
||||
}
|
||||
if (data != null) {
|
||||
CodecUtil.writeFooter(data); // write checksum
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(data, meta);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(data, meta);
|
||||
}
|
||||
meta = data = null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,195 @@
|
|||
package org.apache.lucene.codecs.lucene410;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.SmallFloat;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.packed.DirectWriter;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
|
||||
/**
|
||||
* Lucene 4.10 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.
|
||||
* </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>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>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>
|
||||
* <ol>
|
||||
* <li><a name="dvm" id="dvm"></a>
|
||||
* <p>The DocValues metadata or .dvm file.</p>
|
||||
* <p>For DocValues field, this stores metadata, such as the offset into the
|
||||
* DocValues data (.dvd)</p>
|
||||
* <p>DocValues metadata (.dvm) --> Header,<Entry><sup>NumFields</sup>,Footer</p>
|
||||
* <ul>
|
||||
* <li>Entry --> NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry | SortedNumericEntry</li>
|
||||
* <li>NumericEntry --> GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
|
||||
* <li>GCDNumericEntry --> NumericHeader,MinValue,GCD,BitsPerValue</li>
|
||||
* <li>TableNumericEntry --> NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,BitsPerValue</li>
|
||||
* <li>DeltaNumericEntry --> NumericHeader,MinValue,BitsPerValue</li>
|
||||
* <li>MonotonicNumericEntry --> NumericHeader,PackedVersion,BlockSize</li>
|
||||
* <li>NumericHeader --> FieldNumber,EntryType,NumericType,MissingOffset,DataOffset,Count,EndOffset</li>
|
||||
* <li>BinaryEntry --> FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
|
||||
* <li>FixedBinaryEntry --> BinaryHeader</li>
|
||||
* <li>VariableBinaryEntry --> BinaryHeader,AddressOffset,PackedVersion,BlockSize</li>
|
||||
* <li>PrefixBinaryEntry --> BinaryHeader,AddressInterval,AddressOffset,PackedVersion,BlockSize</li>
|
||||
* <li>BinaryHeader --> FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
|
||||
* <li>SortedEntry --> FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
|
||||
* <li>SortedSetEntry --> EntryType,BinaryEntry,NumericEntry,NumericEntry</li>
|
||||
* <li>SortedNumericEntry --> EntryType,NumericEntry,NumericEntry</li>
|
||||
* <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --> {@link DataOutput#writeVInt VInt}</li>
|
||||
* <li>EntryType,CompressionType --> {@link DataOutput#writeByte Byte}</li>
|
||||
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
|
||||
* <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset,EndOffset --> {@link DataOutput#writeLong Int64}</li>
|
||||
* <li>TableSize,BitsPerValue --> {@link DataOutput#writeVInt vInt}</li>
|
||||
* <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}</li>
|
||||
* </ul>
|
||||
* <p>Sorted fields have two entries: a BinaryEntry with the value metadata,
|
||||
* and an ordinary NumericEntry for the document-to-ord metadata.</p>
|
||||
* <p>SortedSet fields have three entries: a BinaryEntry with the value metadata,
|
||||
* and two NumericEntries for the document-to-ord-index and ordinal list metadata.</p>
|
||||
* <p>SortedNumeric fields have two entries: A NumericEntry with the value metadata,
|
||||
* and a numeric entry with the document-to-value index.</p>
|
||||
* <p>FieldNumber of -1 indicates the end of metadata.</p>
|
||||
* <p>EntryType is a 0 (NumericEntry) or 1 (BinaryEntry)</p>
|
||||
* <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
|
||||
* <p>EndOffset is the pointer to the end of the data in the DocValues data (.dvd)</p>
|
||||
* <p>NumericType indicates how Numeric values will be compressed:
|
||||
* <ul>
|
||||
* <li>0 --> delta-compressed. For each block of 16k integers, every integer is delta-encoded
|
||||
* from the minimum value within the block.
|
||||
* <li>1 -->, gcd-compressed. When all integers share a common divisor, only quotients are stored
|
||||
* using blocks of delta-encoded ints.
|
||||
* <li>2 --> table-compressed. When the number of unique numeric values is small and it would save space,
|
||||
* a lookup table of unique values is written, followed by the ordinal for each document.
|
||||
* </ul>
|
||||
* <p>BinaryType indicates how Binary values will be stored:
|
||||
* <ul>
|
||||
* <li>0 --> fixed-width. All values have the same length, addressing by multiplication.
|
||||
* <li>1 -->, variable-width. An address for each value is stored.
|
||||
* <li>2 --> prefix-compressed. An address to the start of every interval'th value is stored.
|
||||
* </ul>
|
||||
* <p>MinLength and MaxLength represent the min and max byte[] value lengths for Binary values.
|
||||
* If they are equal, then all values are of a fixed size, and can be addressed as DataOffset + (docID * length).
|
||||
* Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
|
||||
* is written for the addresses.
|
||||
* <p>MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
|
||||
* If its -1, then there are no missing values.
|
||||
* <p>Checksum contains the CRC32 checksum of all bytes in the .dvm file up
|
||||
* until the checksum. This is used to verify integrity of the file on opening the
|
||||
* index.
|
||||
* <li><a name="dvd" id="dvd"></a>
|
||||
* <p>The DocValues data or .dvd file.</p>
|
||||
* <p>For DocValues field, this stores the actual per-document data (the heavy-lifting)</p>
|
||||
* <p>DocValues data (.dvd) --> Header,<NumericData | BinaryData | SortedData><sup>NumFields</sup>,Footer</p>
|
||||
* <ul>
|
||||
* <li>NumericData --> DeltaCompressedNumerics | TableCompressedNumerics | GCDCompressedNumerics</li>
|
||||
* <li>BinaryData --> {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
|
||||
* <li>SortedData --> {@link FST FST<Int64>}</li>
|
||||
* <li>DeltaCompressedNumerics,TableCompressedNumerics,GCDCompressedNumerics --> {@link DirectWriter PackedInts}</li>
|
||||
* <li>Addresses --> {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
|
||||
* <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}</li>
|
||||
* </ul>
|
||||
* </ol>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene410DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
/** Sole Constructor */
|
||||
public Lucene410DocValuesFormat() {
|
||||
super("Lucene410");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new Lucene410DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return new Lucene410DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
static final String DATA_CODEC = "Lucene410DocValuesData";
|
||||
static final String DATA_EXTENSION = "dvd";
|
||||
static final String META_CODEC = "Lucene410ValuesMetadata";
|
||||
static final String META_EXTENSION = "dvm";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
static final byte NUMERIC = 0;
|
||||
static final byte BINARY = 1;
|
||||
static final byte SORTED = 2;
|
||||
static final byte SORTED_SET = 3;
|
||||
static final byte SORTED_NUMERIC = 4;
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,404 @@
|
|||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
(the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<html>
|
||||
<head>
|
||||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
||||
</head>
|
||||
<body>
|
||||
Lucene 4.10 file format.
|
||||
|
||||
<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></li>
|
||||
<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>
|
||||
</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>Apache Lucene is written in Java, but several efforts are underway to write
|
||||
<a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
|
||||
Lucene in other programming languages</a>. If these versions are to remain
|
||||
compatible with Apache Lucene, then a language-independent definition of the
|
||||
Lucene index format is required. This document thus attempts to provide a
|
||||
complete and independent definition of the Apache Lucene file formats.</p>
|
||||
<p>As Lucene evolves, this document should evolve. Versions of Lucene in
|
||||
different programming languages should endeavor to agree on file formats, and
|
||||
generate new versions of this document.</p>
|
||||
</div>
|
||||
<a name="Definitions" id="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" id="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" id="Overview"></a>
|
||||
<h2>Index Structure Overview</h2>
|
||||
<div>
|
||||
<p>Each segment index maintains the following:</p>
|
||||
<ul>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene46.Lucene46SegmentInfoFormat 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.lucene46.Lucene46FieldInfosFormat Field names}.
|
||||
This contains the set of field names used in the index.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat 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.lucene41.Lucene41PostingsFormat 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.lucene41.Lucene41PostingsFormat 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.lucene41.Lucene41PostingsFormat 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.lucene49.Lucene49NormsFormat Normalization factors}.
|
||||
For each field in each document, a value is stored
|
||||
that is multiplied into the score for hits on that field.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat 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.lucene410.Lucene410DocValuesFormat 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.lucene40.Lucene40LiveDocsFormat Deleted documents}.
|
||||
An optional file indicating which documents are deleted.
|
||||
</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 in 1.4 and greater) 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>As of version 2.1 (lock-less commits), file names are never re-used (there
|
||||
is one exception, "segments.gen", see below). 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" id="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">
|
||||
<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.gen, 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.lucene40.Lucene40SegmentInfoFormat Segment Info}</td>
|
||||
<td>.si</td>
|
||||
<td>Stores metadata about a segment</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.store.CompoundFileDirectory 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.lucene46.Lucene46FieldInfosFormat Fields}</td>
|
||||
<td>.fnm</td>
|
||||
<td>Stores information about the fields</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Index}</td>
|
||||
<td>.fdx</td>
|
||||
<td>Contains pointers to field data</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Data}</td>
|
||||
<td>.fdt</td>
|
||||
<td>The stored fields for documents</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary}</td>
|
||||
<td>.tim</td>
|
||||
<td>The term dictionary, stores term info</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index}</td>
|
||||
<td>.tip</td>
|
||||
<td>The index into the Term Dictionary</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat 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.lucene41.Lucene41PostingsFormat 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.lucene41.Lucene41PostingsFormat 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.lucene49.Lucene49NormsFormat 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.lucene410.Lucene410DocValuesFormat 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.lucene42.Lucene42TermVectorsFormat Term Vector Index}</td>
|
||||
<td>.tvx</td>
|
||||
<td>Stores offset into the document data file</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Documents}</td>
|
||||
<td>.tvd</td>
|
||||
<td>Contains information about each document that has term vectors</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Fields}</td>
|
||||
<td>.tvf</td>
|
||||
<td>The field level info about term vectors</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted Documents}</td>
|
||||
<td>.del</td>
|
||||
<td>Info about what files are deleted</td>
|
||||
</tr>
|
||||
</table>
|
||||
</div>
|
||||
<a name="Lock_File" id="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.</p>
|
||||
<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>
|
||||
</ul>
|
||||
<a name="Limitations" id="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>
|
||||
</body>
|
||||
</html>
|
|
@ -17,7 +17,10 @@ package org.apache.lucene.codecs.lucene49;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
|
@ -35,6 +38,7 @@ import org.apache.lucene.codecs.lucene46.Lucene46FieldInfosFormat;
|
|||
import org.apache.lucene.codecs.lucene46.Lucene46SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 4.9 index format, with configurable per-field postings
|
||||
|
@ -131,10 +135,15 @@ public class Lucene49Codec extends Codec {
|
|||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene49");
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene49NormsFormat();
|
||||
private final NormsFormat normsFormat = new Lucene49NormsFormat() {
|
||||
@Override
|
||||
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
};
|
||||
|
||||
@Override
|
||||
public final NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
public NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.lucene.codecs.DocValuesConsumer;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -92,6 +93,7 @@ class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
|||
|
||||
@Override
|
||||
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
checkCanWrite(field);
|
||||
addNumericField(field, values, true);
|
||||
}
|
||||
|
||||
|
@ -246,6 +248,7 @@ class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
|||
|
||||
@Override
|
||||
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
checkCanWrite(field);
|
||||
// write the byte[] data
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene49DocValuesFormat.BINARY);
|
||||
|
@ -362,6 +365,7 @@ class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
|||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
checkCanWrite(field);
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene49DocValuesFormat.SORTED);
|
||||
addTermsDict(field, values);
|
||||
|
@ -370,6 +374,7 @@ class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
|||
|
||||
@Override
|
||||
public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
|
||||
checkCanWrite(field);
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene49DocValuesFormat.SORTED_NUMERIC);
|
||||
if (isSingleValued(docToValueCount)) {
|
||||
|
@ -387,6 +392,7 @@ class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
|||
|
||||
@Override
|
||||
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
|
||||
checkCanWrite(field);
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene49DocValuesFormat.SORTED_SET);
|
||||
|
||||
|
@ -452,4 +458,14 @@ class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
|||
meta = data = null;
|
||||
}
|
||||
}
|
||||
|
||||
void checkCanWrite(FieldInfo field) {
|
||||
if ((field.getDocValuesType() == DocValuesType.NUMERIC ||
|
||||
field.getDocValuesType() == DocValuesType.BINARY) &&
|
||||
field.getDocValuesGen() != -1) {
|
||||
// ok
|
||||
} else {
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -164,7 +164,7 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
|||
* </ol>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene49DocValuesFormat extends DocValuesFormat {
|
||||
public class Lucene49DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
/** Sole Constructor */
|
||||
public Lucene49DocValuesFormat() {
|
||||
|
|
|
@ -97,7 +97,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* </ol>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene49NormsFormat extends NormsFormat {
|
||||
public class Lucene49NormsFormat extends NormsFormat {
|
||||
|
||||
/** Sole Constructor */
|
||||
public Lucene49NormsFormat() {}
|
||||
|
|
|
@ -61,13 +61,13 @@ name of your codec.
|
|||
If you just want to customise the {@link org.apache.lucene.codecs.PostingsFormat}, or use different postings
|
||||
formats for different fields, then you can register your custom postings format in the same way (in
|
||||
META-INF/services/org.apache.lucene.codecs.PostingsFormat), and then extend the default
|
||||
{@link org.apache.lucene.codecs.lucene49.Lucene49Codec} and override
|
||||
{@link org.apache.lucene.codecs.lucene49.Lucene49Codec#getPostingsFormatForField(String)} to return your custom
|
||||
{@link org.apache.lucene.codecs.lucene410.Lucene410Codec} and override
|
||||
{@link org.apache.lucene.codecs.lucene410.Lucene410Codec#getPostingsFormatForField(String)} to return your custom
|
||||
postings format.
|
||||
</p>
|
||||
<p>
|
||||
Similarly, if you just want to customise the {@link org.apache.lucene.codecs.DocValuesFormat} per-field, have
|
||||
a look at {@link org.apache.lucene.codecs.lucene49.Lucene49Codec#getDocValuesFormatForField(String)}.
|
||||
a look at {@link org.apache.lucene.codecs.lucene410.Lucene410Codec#getDocValuesFormatForField(String)}.
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
||||
|
|
|
@ -46,6 +46,23 @@ public abstract class StringHelper {
|
|||
return i;
|
||||
return len;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the length of {@code currentTerm} needed for use as a sort key.
|
||||
* so that {@link BytesRef#compareTo(BytesRef)} still returns the same result.
|
||||
* This method assumes currentTerm comes after priorTerm.
|
||||
*/
|
||||
public static int sortKeyLength(final BytesRef priorTerm, final BytesRef currentTerm) {
|
||||
final int currentTermOffset = currentTerm.offset;
|
||||
final int priorTermOffset = priorTerm.offset;
|
||||
final int limit = Math.min(priorTerm.length, currentTerm.length);
|
||||
for (int i = 0; i < limit; i++) {
|
||||
if (priorTerm.bytes[priorTermOffset+i] != currentTerm.bytes[currentTermOffset+i]) {
|
||||
return i+1;
|
||||
}
|
||||
}
|
||||
return Math.min(1+priorTerm.length, currentTerm.length);
|
||||
}
|
||||
|
||||
private StringHelper() {
|
||||
}
|
||||
|
|
|
@ -19,3 +19,4 @@ org.apache.lucene.codecs.lucene42.Lucene42Codec
|
|||
org.apache.lucene.codecs.lucene45.Lucene45Codec
|
||||
org.apache.lucene.codecs.lucene46.Lucene46Codec
|
||||
org.apache.lucene.codecs.lucene49.Lucene49Codec
|
||||
org.apache.lucene.codecs.lucene410.Lucene410Codec
|
||||
|
|
|
@ -16,3 +16,4 @@
|
|||
org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat
|
||||
org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat
|
||||
org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat
|
||||
org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
|
@ -37,7 +37,7 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
|
||||
public class TestExternalCodecs extends LuceneTestCase {
|
||||
|
||||
private static final class CustomPerFieldCodec extends Lucene49Codec {
|
||||
private static final class CustomPerFieldCodec extends Lucene410Codec {
|
||||
|
||||
private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
|
|
|
@ -0,0 +1,269 @@
|
|||
package org.apache.lucene.codecs.lucene410;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Tests Lucene410DocValuesFormat
|
||||
*/
|
||||
public class TestLucene410DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene410DocValuesFormat());
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
// TODO: these big methods can easily blow up some of the other ram-hungry codecs...
|
||||
// for now just keep them here, as we want to test this for this format.
|
||||
|
||||
public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16);
|
||||
}
|
||||
}
|
||||
|
||||
@Nightly
|
||||
public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16);
|
||||
}
|
||||
}
|
||||
|
||||
public void testSortedVariableLengthBigVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedVsStoredFields(atLeast(300), 1, 32766);
|
||||
}
|
||||
}
|
||||
|
||||
@Nightly
|
||||
public void testSortedVariableLengthManyVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
|
||||
}
|
||||
}
|
||||
|
||||
public void testTermsEnumFixedWidth() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
|
||||
}
|
||||
}
|
||||
|
||||
public void testTermsEnumVariableWidth() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
|
||||
}
|
||||
}
|
||||
|
||||
@Nightly
|
||||
public void testTermsEnumRandomMany() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: try to refactor this and some termsenum tests into the base class.
|
||||
// to do this we need to fix the test class to get a DVF not a Codec so we can setup
|
||||
// the postings format correctly.
|
||||
private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
|
||||
Directory dir = newFSDirectory(createTempDir());
|
||||
IndexWriterConfig conf = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
// set to duel against a codec which has ordinals:
|
||||
final PostingsFormat pf = new Lucene41WithOrds();
|
||||
final DocValuesFormat dv = new Lucene410DocValuesFormat();
|
||||
conf.setCodec(new Lucene410Codec() {
|
||||
@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 (AtomicReaderContext context : ir.leaves()) {
|
||||
AtomicReader r = context.reader();
|
||||
Terms terms = r.terms("indexed");
|
||||
if (terms != null) {
|
||||
TermsEnum expected = terms.iterator(null);
|
||||
TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
|
||||
assertEquals(r.maxDoc(), expected, actual);
|
||||
}
|
||||
}
|
||||
ir.close();
|
||||
|
||||
writer.forceMerge(1);
|
||||
|
||||
// now compare again after the merge
|
||||
ir = writer.getReader();
|
||||
AtomicReader ar = getOnlySegmentReader(ir);
|
||||
Terms terms = ar.terms("indexed");
|
||||
if (terms != null) {
|
||||
TermsEnum expected = terms.iterator(null);
|
||||
TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
|
||||
assertEquals(ar.maxDoc(), 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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -19,14 +19,19 @@ package org.apache.lucene.codecs.lucene49;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
/**
|
||||
* Tests Lucene49DocValuesFormat
|
||||
*/
|
||||
public class TestLucene49DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene49DocValuesFormat());
|
||||
|
||||
private final Codec codec = new Lucene49RWCodec();
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() {
|
||||
OLD_FORMAT_IMPERSONATION_IS_ACTIVE = true; // explicitly instantiates ancient codec
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
|
|
|
@ -19,15 +19,21 @@ package org.apache.lucene.codecs.lucene49;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BaseNormsFormatTestCase;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
/**
|
||||
* Tests Lucene49NormsFormat
|
||||
*/
|
||||
public class TestLucene49NormsFormat extends BaseNormsFormatTestCase {
|
||||
final Codec codec = new Lucene49Codec();
|
||||
private final Codec codec = new Lucene49RWCodec();
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() {
|
||||
OLD_FORMAT_IMPERSONATION_IS_ACTIVE = true; // explicitly instantiates ancient codec
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -79,9 +79,9 @@ public class TestPerFieldDocValuesFormat extends BaseDocValuesFormatTestCase {
|
|||
Directory directory = newDirectory();
|
||||
// we don't use RandomIndexWriter because it might add more docvalues than we expect !!!!1
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
|
||||
final DocValuesFormat fast = DocValuesFormat.forName("Lucene49");
|
||||
final DocValuesFormat fast = DocValuesFormat.forName("Lucene410");
|
||||
final DocValuesFormat slow = DocValuesFormat.forName("SimpleText");
|
||||
iwc.setCodec(new Lucene49Codec() {
|
||||
iwc.setCodec(new Lucene410Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
if ("dv1".equals(field)) {
|
||||
|
|
|
@ -22,7 +22,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
|
||||
|
@ -200,7 +200,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
|
||||
}
|
||||
|
||||
public static class MockCodec extends Lucene49Codec {
|
||||
public static class MockCodec extends Lucene410Codec {
|
||||
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
|
||||
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
|
||||
final PostingsFormat memory = new MemoryPostingsFormat();
|
||||
|
@ -217,7 +217,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public static class MockCodec2 extends Lucene49Codec {
|
||||
public static class MockCodec2 extends Lucene410Codec {
|
||||
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
|
||||
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
|
||||
|
||||
|
@ -268,7 +268,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testSameCodecDifferentInstance() throws Exception {
|
||||
Codec codec = new Lucene49Codec() {
|
||||
Codec codec = new Lucene410Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if ("id".equals(field)) {
|
||||
|
@ -284,7 +284,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testSameCodecDifferentParams() throws Exception {
|
||||
Codec codec = new Lucene49Codec() {
|
||||
Codec codec = new Lucene410Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if ("id".equals(field)) {
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -1070,7 +1070,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
aux2.close();
|
||||
}
|
||||
|
||||
private static final class CustomPerFieldCodec extends Lucene49Codec {
|
||||
private static final class CustomPerFieldCodec extends Lucene410Codec {
|
||||
private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
private final PostingsFormat memoryFormat = PostingsFormat.forName("Memory");
|
||||
|
@ -1121,7 +1121,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
private static final class UnRegisteredCodec extends FilterCodec {
|
||||
public UnRegisteredCodec() {
|
||||
super("NotRegistered", new Lucene49Codec());
|
||||
super("NotRegistered", new Lucene410Codec());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
|
@ -39,7 +39,7 @@ public class TestAllFilesHaveChecksumFooter extends LuceneTestCase {
|
|||
public void test() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setCodec(new Lucene49Codec());
|
||||
conf.setCodec(new Lucene410Codec());
|
||||
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
|
||||
Document doc = new Document();
|
||||
// these fields should sometimes get term vectors, etc
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
|
@ -39,7 +39,7 @@ public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
|
|||
public void test() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setCodec(new Lucene49Codec());
|
||||
conf.setCodec(new Lucene410Codec());
|
||||
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
|
||||
Document doc = new Document();
|
||||
// these fields should sometimes get term vectors, etc
|
||||
|
|
|
@ -78,8 +78,7 @@ import org.junit.BeforeClass;
|
|||
// TODO: don't use 4.x codec, its unrealistic since it means
|
||||
// we won't even be running the actual code, only the impostor
|
||||
// @SuppressCodecs("Lucene4x")
|
||||
// Sep codec cannot yet handle the offsets in our 4.x index!
|
||||
@SuppressCodecs({"Lucene40", "Lucene41", "Lucene42", "Lucene45", "Lucene46"})
|
||||
@SuppressCodecs({"Lucene40", "Lucene41", "Lucene42", "Lucene45", "Lucene46", "Lucene49"})
|
||||
public class TestBackwardsCompatibility extends LuceneTestCase {
|
||||
|
||||
// Uncomment these cases & run them on an older Lucene version,
|
||||
|
|
|
@ -16,8 +16,8 @@ import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
|
|||
import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45RWCodec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
|
@ -493,10 +493,10 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
|||
public void testDifferentDVFormatPerField() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setCodec(new Lucene49Codec() {
|
||||
conf.setCodec(new Lucene410Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return new Lucene49DocValuesFormat();
|
||||
return new Lucene410DocValuesFormat();
|
||||
}
|
||||
});
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
|
@ -1061,10 +1061,10 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
|||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setMergePolicy(NoMergePolicy.INSTANCE); // disable merges to simplify test assertions.
|
||||
conf.setCodec(new Lucene49Codec() {
|
||||
conf.setCodec(new Lucene410Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return new Lucene49DocValuesFormat();
|
||||
return new Lucene410DocValuesFormat();
|
||||
}
|
||||
});
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
|
@ -1078,7 +1078,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
|||
// change format
|
||||
conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setMergePolicy(NoMergePolicy.INSTANCE); // disable merges to simplify test assertions.
|
||||
conf.setCodec(new Lucene49Codec() {
|
||||
conf.setCodec(new Lucene410Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return new AssertingDocValuesFormat();
|
||||
|
|
|
@ -626,7 +626,7 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
|
|||
MockDirectoryWrapper dir = newMockDirectory();
|
||||
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(Codec.forName("Lucene49"));
|
||||
iwc.setCodec(Codec.forName("Lucene410"));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("id", "id", Field.Store.NO));
|
||||
|
|
|
@ -50,7 +50,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
|
|||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
// for now its SimpleText vs Lucene49(random postings format)
|
||||
// for now its SimpleText vs Lucene410(random postings format)
|
||||
// as this gives the best overall coverage. when we have more
|
||||
// codecs we should probably pick 2 from Codec.availableCodecs()
|
||||
|
||||
|
|
|
@ -280,7 +280,7 @@ public class TestIndexWriterThreadsToSegments extends LuceneTestCase {
|
|||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setRAMBufferSizeMB(.2);
|
||||
Codec codec = Codec.forName("Lucene49");
|
||||
Codec codec = Codec.forName("Lucene410");
|
||||
iwc.setCodec(codec);
|
||||
iwc.setMergePolicy(NoMergePolicy.INSTANCE);
|
||||
final IndexWriter w = new IndexWriter(dir, iwc);
|
||||
|
|
|
@ -16,8 +16,8 @@ import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
|
|||
import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45RWCodec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
|
@ -479,10 +479,10 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
|||
public void testDifferentDVFormatPerField() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setCodec(new Lucene49Codec() {
|
||||
conf.setCodec(new Lucene410Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return new Lucene49DocValuesFormat();
|
||||
return new Lucene410DocValuesFormat();
|
||||
}
|
||||
});
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
|
@ -1057,10 +1057,10 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
|||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setMergePolicy(NoMergePolicy.INSTANCE); // disable merges to simplify test assertions.
|
||||
conf.setCodec(new Lucene49Codec() {
|
||||
conf.setCodec(new Lucene410Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return new Lucene49DocValuesFormat();
|
||||
return new Lucene410DocValuesFormat();
|
||||
}
|
||||
});
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
|
@ -1074,7 +1074,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
|||
// change format
|
||||
conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setMergePolicy(NoMergePolicy.INSTANCE); // disable merges to simplify test assertions.
|
||||
conf.setCodec(new Lucene49Codec() {
|
||||
conf.setCodec(new Lucene410Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return new AssertingDocValuesFormat();
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.lang.reflect.Field;
|
|||
import java.util.HashMap;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
|
@ -64,7 +64,7 @@ public class TestOrdinalMap extends LuceneTestCase {
|
|||
|
||||
public void testRamBytesUsed() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig cfg = new IndexWriterConfig(new MockAnalyzer(random())).setCodec(TestUtil.alwaysDocValuesFormat(new Lucene49DocValuesFormat()));
|
||||
IndexWriterConfig cfg = new IndexWriterConfig(new MockAnalyzer(random())).setCodec(TestUtil.alwaysDocValuesFormat(new Lucene410DocValuesFormat()));
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, cfg);
|
||||
final int maxDoc = TestUtil.nextInt(random(), 10, 1000);
|
||||
final int maxTermLength = TestUtil.nextInt(random(), 1, 4);
|
||||
|
|
|
@ -220,7 +220,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
|
|||
TieredMergePolicy tmp = (TieredMergePolicy) iwc.getMergePolicy();
|
||||
tmp.setFloorSegmentMB(0.00001);
|
||||
// We need stable sizes for each segment:
|
||||
iwc.setCodec(Codec.forName("Lucene49"));
|
||||
iwc.setCodec(Codec.forName("Lucene410"));
|
||||
iwc.setMergeScheduler(new SerialMergeScheduler());
|
||||
iwc.setMaxBufferedDocs(100);
|
||||
iwc.setRAMBufferSizeMB(-1);
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.search;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectDocValuesFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -44,7 +44,7 @@ public class TestSortedSetSelector extends LuceneTestCase {
|
|||
int victim = random().nextInt(2);
|
||||
switch(victim) {
|
||||
case 0: Codec.setDefault(TestUtil.alwaysDocValuesFormat(new DirectDocValuesFormat())); break;
|
||||
default: Codec.setDefault(TestUtil.alwaysDocValuesFormat(new Lucene49DocValuesFormat()));
|
||||
default: Codec.setDefault(TestUtil.alwaysDocValuesFormat(new Lucene410DocValuesFormat()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -26,8 +26,8 @@ import org.apache.lucene.codecs.Codec;
|
|||
public class TestNamedSPILoader extends LuceneTestCase {
|
||||
|
||||
public void testLookup() {
|
||||
Codec codec = Codec.forName("Lucene49");
|
||||
assertEquals("Lucene49", codec.getName());
|
||||
Codec codec = Codec.forName("Lucene410");
|
||||
assertEquals("Lucene410", codec.getName());
|
||||
}
|
||||
|
||||
// we want an exception if its not found.
|
||||
|
@ -40,6 +40,6 @@ public class TestNamedSPILoader extends LuceneTestCase {
|
|||
|
||||
public void testAvailableServices() {
|
||||
Set<String> codecs = Codec.availableCodecs();
|
||||
assertTrue(codecs.contains("Lucene49"));
|
||||
assertTrue(codecs.contains("Lucene410"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,4 +26,12 @@ public class TestStringHelper extends LuceneTestCase {
|
|||
assertEquals(0x111e7435, StringHelper.murmurhash3_x86_32(new BytesRef("You want weapons? We're in a library! Books! The best weapons in the world!"), 0));
|
||||
assertEquals(0x2c628cd0, StringHelper.murmurhash3_x86_32(new BytesRef("You want weapons? We're in a library! Books! The best weapons in the world!"), 3476));
|
||||
}
|
||||
|
||||
public void testSortKeyLength() throws Exception {
|
||||
assertEquals(3, StringHelper.sortKeyLength(new BytesRef("foo"), new BytesRef("for")));
|
||||
assertEquals(3, StringHelper.sortKeyLength(new BytesRef("foo1234"), new BytesRef("for1234")));
|
||||
assertEquals(2, StringHelper.sortKeyLength(new BytesRef("foo"), new BytesRef("fz")));
|
||||
assertEquals(1, StringHelper.sortKeyLength(new BytesRef("foo"), new BytesRef("g")));
|
||||
assertEquals(4, StringHelper.sortKeyLength(new BytesRef("foo"), new BytesRef("food")));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -342,17 +342,12 @@ public class TestFieldCacheVsDocValues extends LuceneTestCase {
|
|||
Document doc = new Document();
|
||||
Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
|
||||
doc.add(idField);
|
||||
final int length;
|
||||
if (minLength == maxLength) {
|
||||
length = minLength; // fixed length
|
||||
} else {
|
||||
length = TestUtil.nextInt(random(), minLength, maxLength);
|
||||
}
|
||||
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(), length));
|
||||
values.add(TestUtil.randomSimpleString(random(), minLength, length));
|
||||
}
|
||||
|
||||
// add in any order to the indexed field
|
||||
|
|
|
@ -21,7 +21,7 @@ import com.spatial4j.core.context.SpatialContext;
|
|||
import com.spatial4j.core.shape.Point;
|
||||
import com.spatial4j.core.shape.Rectangle;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
|
@ -84,7 +84,7 @@ public abstract class SpatialTestCase extends LuceneTestCase {
|
|||
final IndexWriterConfig indexWriterConfig = LuceneTestCase.newIndexWriterConfig(random, new MockAnalyzer(random));
|
||||
//TODO can we randomly choose a doc-values supported format?
|
||||
if (needsDocValues())
|
||||
indexWriterConfig.setCodec( TestUtil.alwaysDocValuesFormat(new Lucene49DocValuesFormat()));;
|
||||
indexWriterConfig.setCodec( TestUtil.alwaysDocValuesFormat(new Lucene410DocValuesFormat()));;
|
||||
return indexWriterConfig;
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
|
|||
import org.apache.lucene.analysis.ngram.Lucene43EdgeNGramTokenFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -185,7 +185,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
* codec to use. */
|
||||
protected IndexWriterConfig getIndexWriterConfig(Analyzer indexAnalyzer, IndexWriterConfig.OpenMode openMode) {
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(indexAnalyzer);
|
||||
iwc.setCodec(new Lucene49Codec());
|
||||
iwc.setCodec(new Lucene410Codec());
|
||||
iwc.setOpenMode(openMode);
|
||||
|
||||
// This way all merged segments will be sorted at
|
||||
|
|
|
@ -23,10 +23,10 @@ import org.apache.lucene.codecs.NormsFormat;
|
|||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
|
||||
/**
|
||||
* Acts like {@link Lucene49Codec} but with additional asserts.
|
||||
* Acts like {@link Lucene410Codec} but with additional asserts.
|
||||
*/
|
||||
public final class AssertingCodec extends FilterCodec {
|
||||
|
||||
|
@ -37,7 +37,7 @@ public final class AssertingCodec extends FilterCodec {
|
|||
private final NormsFormat norms = new AssertingNormsFormat();
|
||||
|
||||
public AssertingCodec() {
|
||||
super("Asserting", new Lucene49Codec());
|
||||
super("Asserting", new Lucene410Codec());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.NoSuchElementException;
|
|||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.index.AssertingAtomicReader;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -40,10 +40,10 @@ import org.apache.lucene.util.FixedBitSet;
|
|||
import org.apache.lucene.util.LongBitSet;
|
||||
|
||||
/**
|
||||
* Just like {@link Lucene49DocValuesFormat} but with additional asserts.
|
||||
* Just like {@link Lucene410DocValuesFormat} but with additional asserts.
|
||||
*/
|
||||
public class AssertingDocValuesFormat extends DocValuesFormat {
|
||||
private final DocValuesFormat in = new Lucene49DocValuesFormat();
|
||||
private final DocValuesFormat in = new Lucene410DocValuesFormat();
|
||||
|
||||
public AssertingDocValuesFormat() {
|
||||
super("Asserting");
|
||||
|
|
|
@ -26,8 +26,8 @@ import org.apache.lucene.codecs.TermVectorsFormat;
|
|||
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
|
||||
|
||||
/** Codec that tries to use as little ram as possible because he spent all his money on beer */
|
||||
|
@ -40,11 +40,11 @@ public class CheapBastardCodec extends FilterCodec {
|
|||
// uncompressing versions, waste lots of disk but no ram
|
||||
private final StoredFieldsFormat storedFields = new Lucene40StoredFieldsFormat();
|
||||
private final TermVectorsFormat termVectors = new Lucene40TermVectorsFormat();
|
||||
private final DocValuesFormat docValues = new Lucene49DocValuesFormat();
|
||||
private final DocValuesFormat docValues = new Lucene410DocValuesFormat();
|
||||
private final NormsFormat norms = new Lucene49NormsFormat();
|
||||
|
||||
public CheapBastardCodec() {
|
||||
super("CheapBastard", new Lucene49Codec());
|
||||
super("CheapBastard", new Lucene410Codec());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,13 +23,13 @@ import org.apache.lucene.codecs.FilterCodec;
|
|||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
||||
|
||||
/**
|
||||
* A codec that uses {@link CompressingStoredFieldsFormat} for its stored
|
||||
* fields and delegates to {@link Lucene49Codec} for everything else.
|
||||
* fields and delegates to {@link Lucene410Codec} for everything else.
|
||||
*/
|
||||
public abstract class CompressingCodec extends FilterCodec {
|
||||
|
||||
|
@ -73,7 +73,7 @@ public abstract class CompressingCodec extends FilterCodec {
|
|||
* Creates a compressing codec with a given segment suffix
|
||||
*/
|
||||
public CompressingCodec(String name, String segmentSuffix, CompressionMode compressionMode, int chunkSize) {
|
||||
super(name, new Lucene49Codec());
|
||||
super(name, new Lucene410Codec());
|
||||
this.storedFieldsFormat = new CompressingStoredFieldsFormat(name, segmentSuffix, compressionMode, chunkSize);
|
||||
this.termVectorsFormat = new CompressingTermVectorsFormat(name, segmentSuffix, compressionMode, chunkSize);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
package org.apache.lucene.codecs.lucene49;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
|
||||
/**
|
||||
* Read-write version of {@link Lucene49Codec} for testing.
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
public class Lucene49RWCodec extends Lucene49Codec {
|
||||
|
||||
private static final DocValuesFormat docValues = new Lucene49RWDocValuesFormat();
|
||||
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
private static final NormsFormat norms = new Lucene49NormsFormat();
|
||||
|
||||
@Override
|
||||
public NormsFormat normsFormat() {
|
||||
return norms;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
package org.apache.lucene.codecs.lucene49;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/** Read-write version of {@link Lucene49DocValuesFormat} for testing */
|
||||
public class Lucene49RWDocValuesFormat extends Lucene49DocValuesFormat {
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
if (LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
|
||||
return new Lucene49DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION) {
|
||||
@Override
|
||||
void checkCanWrite(FieldInfo field) {
|
||||
// allow writing all fields
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return super.fieldsConsumer(state);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,25 @@
|
|||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
(the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<html>
|
||||
<head>
|
||||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
||||
</head>
|
||||
<body>
|
||||
Support for testing {@link org.apache.lucene.codecs.lucene49.Lucene49Codec}.
|
||||
</body>
|
||||
</html>
|
|
@ -818,6 +818,8 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
assertEquals("beer", termsEnum.term().utf8ToString());
|
||||
assertEquals(0, termsEnum.ord());
|
||||
assertEquals(SeekStatus.END, termsEnum.seekCeil(new BytesRef("zzz")));
|
||||
assertEquals(SeekStatus.NOT_FOUND, termsEnum.seekCeil(new BytesRef("aba")));
|
||||
assertEquals(0, termsEnum.ord());
|
||||
|
||||
// seekExact()
|
||||
assertTrue(termsEnum.seekExact(new BytesRef("beer")));
|
||||
|
@ -1383,8 +1385,8 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
}
|
||||
}
|
||||
|
||||
private void doTestSortedVsStoredFields(int minLength, int maxLength) throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
protected void doTestSortedVsStoredFields(int numDocs, int minLength, int maxLength) throws Exception {
|
||||
Directory dir = newFSDirectory(createTempDir("dvduel"));
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
|
||||
Document doc = new Document();
|
||||
|
@ -1396,7 +1398,6 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
doc.add(dvField);
|
||||
|
||||
// index some docs
|
||||
int numDocs = atLeast(300);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
idField.setStringValue(Integer.toString(i));
|
||||
final int length;
|
||||
|
@ -1456,14 +1457,14 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
int fixedLength = TestUtil.nextInt(random(), 1, 10);
|
||||
doTestSortedVsStoredFields(fixedLength, fixedLength);
|
||||
doTestSortedVsStoredFields(atLeast(300), fixedLength, fixedLength);
|
||||
}
|
||||
}
|
||||
|
||||
public void testSortedVariableLengthVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedVsStoredFields(1, 10);
|
||||
doTestSortedVsStoredFields(atLeast(300), 1, 10);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1905,13 +1906,12 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
directory.close();
|
||||
}
|
||||
|
||||
private void doTestSortedSetVsStoredFields(int minLength, int maxLength, int maxValuesPerDoc) throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
protected void doTestSortedSetVsStoredFields(int numDocs, int minLength, int maxLength, int maxValuesPerDoc) throws Exception {
|
||||
Directory dir = newFSDirectory(createTempDir("dvduel"));
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
|
||||
|
||||
// index some docs
|
||||
int numDocs = atLeast(300);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document doc = new Document();
|
||||
Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
|
||||
|
@ -2007,7 +2007,7 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
int fixedLength = TestUtil.nextInt(random(), 1, 10);
|
||||
doTestSortedSetVsStoredFields(fixedLength, fixedLength, 16);
|
||||
doTestSortedSetVsStoredFields(atLeast(300), fixedLength, fixedLength, 16);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2078,7 +2078,7 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedSetVsStoredFields(1, 10, 16);
|
||||
doTestSortedSetVsStoredFields(atLeast(300), 1, 10, 16);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2087,7 +2087,7 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
int fixedLength = TestUtil.nextInt(random(), 1, 10);
|
||||
doTestSortedSetVsStoredFields(fixedLength, fixedLength, 1);
|
||||
doTestSortedSetVsStoredFields(atLeast(300), fixedLength, fixedLength, 1);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2095,7 +2095,7 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
|||
assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedSetVsStoredFields(1, 10, 1);
|
||||
doTestSortedSetVsStoredFields(atLeast(300), 1, 10, 1);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -1519,7 +1519,7 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
|
|||
// TODO: would be better to use / delegate to the current
|
||||
// Codec returned by getCodec()
|
||||
|
||||
iwc.setCodec(new Lucene49Codec() {
|
||||
iwc.setCodec(new Lucene410Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
|
||||
|
|
|
@ -32,7 +32,7 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoubleField;
|
||||
|
@ -491,7 +491,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends BaseIndexFileFormat
|
|||
// get another codec, other than the default: so we are merging segments across different codecs
|
||||
final Codec otherCodec;
|
||||
if ("SimpleText".equals(Codec.getDefault().getName())) {
|
||||
otherCodec = new Lucene49Codec();
|
||||
otherCodec = new Lucene410Codec();
|
||||
} else {
|
||||
otherCodec = new SimpleTextCodec();
|
||||
}
|
||||
|
|
|
@ -36,8 +36,8 @@ import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
|||
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
|
||||
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
|
||||
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat;
|
||||
|
@ -62,7 +62,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
* documents in different orders and the test will still be deterministic
|
||||
* and reproducable.
|
||||
*/
|
||||
public class RandomCodec extends Lucene49Codec {
|
||||
public class RandomCodec extends Lucene410Codec {
|
||||
/** Shuffled list of postings formats to use for new mappings */
|
||||
private List<PostingsFormat> formats = new ArrayList<>();
|
||||
|
||||
|
@ -152,7 +152,7 @@ public class RandomCodec extends Lucene49Codec {
|
|||
new MemoryPostingsFormat(false, random.nextFloat()));
|
||||
|
||||
addDocValues(avoidCodecs,
|
||||
new Lucene49DocValuesFormat(),
|
||||
new Lucene410DocValuesFormat(),
|
||||
new MemoryDocValuesFormat(),
|
||||
new SimpleTextDocValuesFormat(),
|
||||
new AssertingDocValuesFormat());
|
||||
|
|
|
@ -37,10 +37,11 @@ import org.apache.lucene.codecs.compressing.CompressingCodec;
|
|||
import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45RWCodec;
|
||||
import org.apache.lucene.codecs.lucene46.Lucene46RWCodec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49RWCodec;
|
||||
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.index.RandomCodec;
|
||||
|
@ -158,7 +159,7 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE = false;
|
||||
|
||||
savedCodec = Codec.getDefault();
|
||||
int randomVal = random.nextInt(10);
|
||||
int randomVal = random.nextInt(11);
|
||||
if ("Lucene40".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
|
||||
"random".equals(TEST_POSTINGSFORMAT) &&
|
||||
"random".equals(TEST_DOCVALUESFORMAT) &&
|
||||
|
@ -200,6 +201,14 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
codec = Codec.forName("Lucene46");
|
||||
LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE = true;
|
||||
assert codec instanceof Lucene46RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
|
||||
} else if ("Lucene49".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
|
||||
"random".equals(TEST_POSTINGSFORMAT) &&
|
||||
"random".equals(TEST_DOCVALUESFORMAT) &&
|
||||
randomVal == 5 &&
|
||||
!shouldAvoidCodec("Lucene49"))) {
|
||||
codec = Codec.forName("Lucene49");
|
||||
LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE = true;
|
||||
assert codec instanceof Lucene49RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
|
||||
} else if (("random".equals(TEST_POSTINGSFORMAT) == false) || ("random".equals(TEST_DOCVALUESFORMAT) == false)) {
|
||||
// the user wired postings or DV: this is messy
|
||||
// refactor into RandomCodec....
|
||||
|
@ -216,12 +225,12 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
final DocValuesFormat dvFormat;
|
||||
if ("random".equals(TEST_DOCVALUESFORMAT)) {
|
||||
// TODO: huh?
|
||||
dvFormat = DocValuesFormat.forName("Lucene49");
|
||||
dvFormat = DocValuesFormat.forName("Lucene410");
|
||||
} else {
|
||||
dvFormat = DocValuesFormat.forName(TEST_DOCVALUESFORMAT);
|
||||
}
|
||||
|
||||
codec = new Lucene49Codec() {
|
||||
codec = new Lucene410Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return format;
|
||||
|
|
|
@ -45,7 +45,7 @@ import java.util.zip.ZipFile;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
|
@ -678,7 +678,7 @@ public final class TestUtil {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("forcing postings format to:" + format);
|
||||
}
|
||||
return new Lucene49Codec() {
|
||||
return new Lucene410Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return format;
|
||||
|
@ -696,7 +696,7 @@ public final class TestUtil {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("forcing docvalues format to:" + format);
|
||||
}
|
||||
return new Lucene49Codec() {
|
||||
return new Lucene410Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return format;
|
||||
|
|
|
@ -24,3 +24,4 @@ org.apache.lucene.codecs.lucene41.Lucene41RWCodec
|
|||
org.apache.lucene.codecs.lucene42.Lucene42RWCodec
|
||||
org.apache.lucene.codecs.lucene45.Lucene45RWCodec
|
||||
org.apache.lucene.codecs.lucene46.Lucene46RWCodec
|
||||
org.apache.lucene.codecs.lucene49.Lucene49RWCodec
|
||||
|
|
|
@ -3,7 +3,7 @@ package org.apache.solr.core;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.util.plugin.SolrCoreAware;
|
||||
|
@ -51,7 +51,7 @@ public class SchemaCodecFactory extends CodecFactory implements SolrCoreAware {
|
|||
@Override
|
||||
public void init(NamedList args) {
|
||||
super.init(args);
|
||||
codec = new Lucene49Codec() {
|
||||
codec = new Lucene410Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);
|
||||
|
|
|
@ -21,7 +21,7 @@
|
|||
<fieldType name="string_simpletext" class="solr.StrField" postingsFormat="SimpleText"/>
|
||||
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene41"/>
|
||||
|
||||
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene49" />
|
||||
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene410" />
|
||||
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory" />
|
||||
|
||||
<fieldType name="string" class="solr.StrField" />
|
||||
|
|
|
@ -53,12 +53,12 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
Map<String, SchemaField> fields = h.getCore().getLatestSchema().getFields();
|
||||
SchemaField schemaField = fields.get("string_disk_f");
|
||||
PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
|
||||
assertEquals("Lucene49", format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
assertEquals("Lucene410", format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_memory_f");
|
||||
assertEquals("Memory",
|
||||
format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_f");
|
||||
assertEquals("Lucene49",
|
||||
assertEquals("Lucene410",
|
||||
format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
}
|
||||
|
||||
|
@ -78,8 +78,8 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
Codec codec = h.getCore().getCodec();
|
||||
PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
|
||||
|
||||
assertEquals("Lucene49", format.getDocValuesFormatForField("foo_disk").getName());
|
||||
assertEquals("Lucene49", format.getDocValuesFormatForField("bar_disk").getName());
|
||||
assertEquals("Lucene410", format.getDocValuesFormatForField("foo_disk").getName());
|
||||
assertEquals("Lucene410", format.getDocValuesFormatForField("bar_disk").getName());
|
||||
assertEquals("Memory", format.getDocValuesFormatForField("foo_memory").getName());
|
||||
assertEquals("Memory", format.getDocValuesFormatForField("bar_memory").getName());
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue