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:
Robert Muir 2014-08-14 15:30:44 +00:00
parent b0f63241dc
commit 99c23632fa
59 changed files with 2941 additions and 130 deletions

View File

@ -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

View File

@ -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;

View File

@ -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());
]]>

View File

@ -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 {

View File

@ -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.

View File

@ -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() {

View File

@ -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;
}
}

View File

@ -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;
}
}
}

View File

@ -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 (&lt; 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) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
* <ul>
* <li>Entry --&gt; NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry | SortedNumericEntry</li>
* <li>NumericEntry --&gt; GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
* <li>GCDNumericEntry --&gt; NumericHeader,MinValue,GCD,BitsPerValue</li>
* <li>TableNumericEntry --&gt; NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,BitsPerValue</li>
* <li>DeltaNumericEntry --&gt; NumericHeader,MinValue,BitsPerValue</li>
* <li>MonotonicNumericEntry --&gt; NumericHeader,PackedVersion,BlockSize</li>
* <li>NumericHeader --&gt; FieldNumber,EntryType,NumericType,MissingOffset,DataOffset,Count,EndOffset</li>
* <li>BinaryEntry --&gt; FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
* <li>FixedBinaryEntry --&gt; BinaryHeader</li>
* <li>VariableBinaryEntry --&gt; BinaryHeader,AddressOffset,PackedVersion,BlockSize</li>
* <li>PrefixBinaryEntry --&gt; BinaryHeader,AddressInterval,AddressOffset,PackedVersion,BlockSize</li>
* <li>BinaryHeader --&gt; FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
* <li>SortedEntry --&gt; FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
* <li>SortedSetEntry --&gt; EntryType,BinaryEntry,NumericEntry,NumericEntry</li>
* <li>SortedNumericEntry --&gt; EntryType,NumericEntry,NumericEntry</li>
* <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset,EndOffset --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>TableSize,BitsPerValue --&gt; {@link DataOutput#writeVInt vInt}</li>
* <li>Footer --&gt; {@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 --&gt; delta-compressed. For each block of 16k integers, every integer is delta-encoded
* from the minimum value within the block.
* <li>1 --&gt, gcd-compressed. When all integers share a common divisor, only quotients are stored
* using blocks of delta-encoded ints.
* <li>2 --&gt; 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 --&gt; fixed-width. All values have the same length, addressing by multiplication.
* <li>1 --&gt, variable-width. An address for each value is stored.
* <li>2 --&gt; 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) --&gt; Header,&lt;NumericData | BinaryData | SortedData&gt;<sup>NumFields</sup>,Footer</p>
* <ul>
* <li>NumericData --&gt; DeltaCompressedNumerics | TableCompressedNumerics | GCDCompressedNumerics</li>
* <li>BinaryData --&gt; {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
* <li>SortedData --&gt; {@link FST FST&lt;Int64&gt;}</li>
* <li>DeltaCompressedNumerics,TableCompressedNumerics,GCDCompressedNumerics --&gt; {@link DirectWriter PackedInts}</li>
* <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
* <li>Footer --&gt; {@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;
}

View File

@ -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 &amp; 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&lt;String,String&gt; 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>

View File

@ -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;
}
}

View File

@ -28,6 +28,7 @@ import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
@ -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");
}
}
}

View File

@ -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() {

View File

@ -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() {}

View File

@ -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>

View File

@ -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() {
}

View File

@ -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

View File

@ -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

View File

@ -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");

View File

@ -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());
}
}
}
}

View File

@ -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;

View File

@ -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;
}
}
}

View File

@ -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)) {

View File

@ -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)) {

View File

@ -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());
}
}

View File

@ -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

View File

@ -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

View File

@ -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,

View File

@ -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();

View File

@ -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));

View File

@ -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()

View File

@ -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);

View File

@ -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();

View File

@ -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);

View File

@ -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);

View File

@ -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()));
}
}

View File

@ -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"));
}
}

View File

@ -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")));
}
}

View File

@ -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

View File

@ -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;
}

View File

@ -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

View File

@ -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

View File

@ -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");

View File

@ -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

View File

@ -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);
}

View File

@ -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;
}
}

View File

@ -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);
}
}
}

View File

@ -0,0 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Support for testing {@link org.apache.lucene.codecs.lucene49.Lucene49Codec}.
</body>
</html>

View File

@ -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);
}
}

View File

@ -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) {

View File

@ -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();
}

View File

@ -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());

View File

@ -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;

View File

@ -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;

View File

@ -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

View File

@ -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);

View File

@ -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" />

View File

@ -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());
}