LUCENE-5731: split out direct packed ints from in-ram ones

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1600412 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2014-06-04 18:26:04 +00:00
parent d330799888
commit 9e6c1432a6
67 changed files with 3385 additions and 453 deletions

View File

@ -117,6 +117,11 @@ New Features
filtered and constant-score queries to postings highlighter.
(Luca Cavanna via Robert Muir)
* LUCENE-5731: Add RandomAccessInput, a random access API for directory.
Add DirectReader/Writer, optimized for reading packed integers directly
from Directory. Add Lucene49Codec and Lucene49DocValuesFormat that make
use of these. (Robert Muir)
Changes in Backwards Compatibility Policy
* LUCENE-5634: Add reuse argument to IndexableField.tokenStream. This

View File

@ -287,7 +287,7 @@
<classpath>
<path refid="groovy.classpath"/>
</classpath><![CDATA[
// private static Codec defaultCodec = Codec . forName ( "Lucene46" ) ;
// private static Codec defaultCodec = Codec . forName ( "Lucene49" ) ;
defaultCodecMatcher = self.getToken() =~ /defaultCodec\s*=\s*Codec\s*\.\s*forName\s*\(\s*"([^"]+)"\s*\)\s*;/
self.setToken(defaultCodecMatcher[0][1].toLowerCase());
]]>

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -43,7 +43,7 @@ public final class DiskDocValuesFormat extends DocValuesFormat {
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION) {
return new Lucene49DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION) {
@Override
protected void addTermsDict(FieldInfo field, Iterable<BytesRef> values) throws IOException {
addBinaryField(field, values);

View File

@ -19,13 +19,13 @@ package org.apache.lucene.codecs.diskdv;
import java.io.IOException;
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesProducer;
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
class DiskDocValuesProducer extends Lucene45DocValuesProducer {
class DiskDocValuesProducer extends Lucene49DocValuesProducer {
DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
super(state, dataCodec, dataExtension, metaCodec, metaExtension);

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -31,7 +31,7 @@ public final class DiskNormsFormat extends NormsFormat {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
return new Lucene49DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override

View File

@ -182,7 +182,7 @@ class MemoryDocValuesConsumer extends DocValuesConsumer {
data.writeLong(gcd);
data.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE, PackedInts.DEFAULT);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
long value = nv == null ? 0 : nv.longValue();
writer.add((value - minValue) / gcd);
@ -194,7 +194,7 @@ class MemoryDocValuesConsumer extends DocValuesConsumer {
meta.writeVInt(PackedInts.VERSION_CURRENT);
data.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE, PackedInts.DEFAULT);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
writer.add(nv == null ? 0 : nv.longValue());
}

View File

@ -119,7 +119,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
loader.reload(classloader);
}
private static Codec defaultCodec = Codec.forName("Lucene46");
private static Codec defaultCodec = Codec.forName("Lucene49");
/** 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 Lucene46's
* codec. For example, if you want to build a codec that redefines Lucene49's
* {@link LiveDocsFormat}:
* <pre class="prettyprint">
* public final class CustomCodec extends FilterCodec {
*
* public CustomCodec() {
* super("CustomCodec", new Lucene46Codec());
* super("CustomCodec", new Lucene49Codec());
* }
*
* public LiveDocsFormat liveDocsFormat() {

View File

@ -241,7 +241,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
vectorsStream.writeVInt(PackedInts.VERSION_CURRENT);
vectorsStream.writeVInt(chunkSize);
writer = new BlockPackedWriter(vectorsStream, BLOCK_SIZE, PackedInts.COMPACT);
writer = new BlockPackedWriter(vectorsStream, BLOCK_SIZE);
positionsBuf = new int[1024];
startOffsetsBuf = new int[1024];

View File

@ -153,7 +153,7 @@ class Lucene42NormsConsumer extends DocValuesConsumer {
data.writeLong(gcd);
data.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE, PackedInts.DEFAULT);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
long value = nv == null ? 0 : nv.longValue();
writer.add((value - minValue) / gcd);
@ -165,7 +165,7 @@ class Lucene42NormsConsumer extends DocValuesConsumer {
meta.writeVInt(PackedInts.VERSION_CURRENT);
data.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE, PackedInts.DEFAULT);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
writer.add(nv == null ? 0 : nv.longValue());
}

View File

@ -17,7 +17,10 @@ package org.apache.lucene.codecs.lucene45;
* 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.lucene42.Lucene42NormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.index.SegmentWriteState;
/**
* Implements the Lucene 4.5 index format, with configurable per-field postings
@ -134,10 +138,15 @@ public class Lucene45Codec extends Codec {
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene45");
private final NormsFormat normsFormat = new Lucene42NormsFormat();
private final NormsFormat normsFormat = new Lucene42NormsFormat() {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};
@Override
public final NormsFormat normsFormat() {
public NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -39,7 +39,7 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
/** writer for {@link Lucene45DocValuesFormat} */
public class Lucene45DocValuesConsumer extends DocValuesConsumer implements Closeable {
class Lucene45DocValuesConsumer extends DocValuesConsumer implements Closeable {
static final int BLOCK_SIZE = 16384;
static final int ADDRESS_INTERVAL = 16;
@ -51,8 +51,6 @@ public class Lucene45DocValuesConsumer extends DocValuesConsumer implements Clos
public static final int GCD_COMPRESSED = 1;
/** Compressed by giving IDs to unique values. */
public static final int TABLE_COMPRESSED = 2;
/** Compressed using just bitpacked integers */
public static final int BITPACK_COMPRESSED = 3;
/** Uncompressed binary, written directly (fixed length). */
public static final int BINARY_FIXED_UNCOMPRESSED = 0;
@ -101,7 +99,6 @@ public class Lucene45DocValuesConsumer extends DocValuesConsumer implements Clos
long maxValue = Long.MIN_VALUE;
long gcd = 0;
boolean missing = false;
boolean block = true;
// TODO: more efficient?
HashSet<Long> uniqueValues = null;
if (optimizeStorage) {
@ -141,19 +138,9 @@ public class Lucene45DocValuesConsumer extends DocValuesConsumer implements Clos
++count;
}
} else {
for (Number nv : values) {
long value = nv.longValue();
assert value >= -1;
minValue = Math.min(minValue, value);
maxValue = Math.max(maxValue, value);
for (@SuppressWarnings("unused") Number nv : values) {
++count;
}
// packed ints doesnt support valueCount > maxValue, and
// we must represent missing ordinal (-1)
if (count < Integer.MAX_VALUE && maxValue < Long.MAX_VALUE) {
block = false;
}
}
final long delta = maxValue - minValue;
@ -165,8 +152,6 @@ public class Lucene45DocValuesConsumer extends DocValuesConsumer implements Clos
format = TABLE_COMPRESSED;
} else if (gcd != 0 && gcd != 1) {
format = GCD_COMPRESSED;
} else if (block == false) {
format = BITPACK_COMPRESSED;
} else {
format = DELTA_COMPRESSED;
}
@ -188,7 +173,7 @@ public class Lucene45DocValuesConsumer extends DocValuesConsumer implements Clos
case GCD_COMPRESSED:
meta.writeLong(minValue);
meta.writeLong(gcd);
final BlockPackedWriter quotientWriter = new BlockPackedWriter(data, BLOCK_SIZE, PackedInts.DEFAULT);
final BlockPackedWriter quotientWriter = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
long value = nv == null ? 0 : nv.longValue();
quotientWriter.add((value - minValue) / gcd);
@ -196,7 +181,7 @@ public class Lucene45DocValuesConsumer extends DocValuesConsumer implements Clos
quotientWriter.finish();
break;
case DELTA_COMPRESSED:
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE, PackedInts.DEFAULT);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
writer.add(nv == null ? 0 : nv.longValue());
}
@ -217,18 +202,6 @@ public class Lucene45DocValuesConsumer extends DocValuesConsumer implements Clos
}
ordsWriter.finish();
break;
case BITPACK_COMPRESSED:
assert count > 0 && count < Integer.MAX_VALUE;
assert maxValue >= -1 && maxValue < Long.MAX_VALUE : maxValue;
int bpv = PackedInts.bitsRequired(maxValue+1);
bpv = PackedInts.fastestDirectBits(bpv, PackedInts.DEFAULT);
meta.writeVInt(bpv);
final PackedInts.Writer bitWriter = PackedInts.getWriterNoHeader(data, PackedInts.Format.PACKED, (int) count, bpv, PackedInts.DEFAULT_BUFFER_SIZE);
for (Number nv : values) {
bitWriter.add(nv.longValue()+1);
}
bitWriter.finish();
break;
default:
throw new AssertionError();
}

View File

@ -50,9 +50,6 @@ import org.apache.lucene.util.packed.PackedInts;
* to this table, and those ordinals are compressed with bitpacking ({@link PackedInts}).
* <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>Bitpack-compressed: per-document integers written as a block for the entire segment.
* This technique will only be used when numbers range from {@code -1 .. Long.MAX_VALUE-1},
* when the blocking for the delta-compressed method would not provide additional compression.
* </ul>
* <p>
* {@link DocValuesType#BINARY BINARY}:
@ -99,7 +96,6 @@ import org.apache.lucene.util.packed.PackedInts;
* <li>GCDNumericEntry --&gt; NumericHeader,MinValue,GCD</li>
* <li>TableNumericEntry --&gt; NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup></li>
* <li>DeltaNumericEntry --&gt; NumericHeader</li>
* <li>DeltaNumericEntry --&gt; NumericHeader,BitsPerValue</li>
* <li>NumericHeader --&gt; FieldNumber,EntryType,NumericType,MissingOffset,PackedVersion,DataOffset,Count,BlockSize</li>
* <li>BinaryEntry --&gt; FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
* <li>FixedBinaryEntry --&gt; BinaryHeader</li>
@ -112,7 +108,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>BitsPerValue,TableSize --&gt; {@link DataOutput#writeVInt vInt}</li>
* <li>TableSize --&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,
@ -126,17 +122,15 @@ import org.apache.lucene.util.packed.PackedInts;
* <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
* <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.
* <li>3 --&gt; bitpack-compressed. When the delta method would not save space, every integer is
* delta encoded from {@code -1} for the entire segment.
* </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>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.
@ -165,9 +159,11 @@ import org.apache.lucene.util.packed.PackedInts;
* <p>SortedSet entries store the list of ordinals in their BinaryData as a
* sequences of increasing {@link DataOutput#writeVLong vLong}s, delta-encoded.</p>
* </ol>
* @deprecated Only for reading old 4.3-4.5 segments
* @lucene.experimental
*/
public final class Lucene45DocValuesFormat extends DocValuesFormat {
@Deprecated
public class Lucene45DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */
public Lucene45DocValuesFormat() {
@ -176,6 +172,7 @@ public final class Lucene45DocValuesFormat extends DocValuesFormat {
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
// really we should be read-only, but to allow posting of dv updates against old segments...
return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@ -191,8 +188,7 @@ public final class Lucene45DocValuesFormat extends DocValuesFormat {
static final int VERSION_START = 0;
static final int VERSION_SORTED_SET_SINGLE_VALUE_OPTIMIZED = 1;
static final int VERSION_CHECKSUM = 2;
static final int VERSION_BITPACK_COMPRESSED = 3;
static final int VERSION_CURRENT = VERSION_BITPACK_COMPRESSED;
static final int VERSION_CURRENT = VERSION_CHECKSUM;
static final byte NUMERIC = 0;
static final byte BINARY = 1;
static final byte SORTED = 2;

View File

@ -25,7 +25,6 @@ import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.GCD_CO
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.SORTED_SET_SINGLE_VALUED_SORTED;
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.SORTED_SET_WITH_ADDRESSES;
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.TABLE_COMPRESSED;
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BITPACK_COMPRESSED;
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat.VERSION_SORTED_SET_SINGLE_VALUE_OPTIMIZED;
import java.io.Closeable; // javadocs
@ -265,9 +264,6 @@ public class Lucene45DocValuesProducer extends DocValuesProducer implements Clos
entry.table[i] = meta.readLong();
}
break;
case BITPACK_COMPRESSED:
entry.bitsRequired = meta.readVInt();
break;
case DELTA_COMPRESSED:
break;
default:
@ -343,14 +339,6 @@ public class Lucene45DocValuesProducer extends DocValuesProducer implements Clos
case DELTA_COMPRESSED:
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
return reader;
case BITPACK_COMPRESSED:
final PackedInts.Reader bits = PackedInts.getDirectReaderNoHeader(data, PackedInts.Format.PACKED, entry.packedIntsVersion, (int) entry.count, entry.bitsRequired);
return new LongValues() {
@Override
public long get(long id) {
return bits.get((int) id) - 1;
}
};
case GCD_COMPRESSED:
final long min = entry.minValue;
final long mult = entry.gcd;
@ -496,7 +484,10 @@ public class Lucene45DocValuesProducer extends DocValuesProducer implements Clos
final int valueCount = (int) binaries.get(field.number).count;
final BinaryDocValues binary = getBinary(field);
NumericEntry entry = ords.get(field.number);
final LongValues ordinals = getNumeric(entry);
IndexInput data = this.data.clone();
data.seek(entry.offset);
final BlockPackedReader ordinals = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
return new SortedDocValues() {
@Override
@ -695,8 +686,6 @@ public class Lucene45DocValuesProducer extends DocValuesProducer implements Clos
/** packed ints blocksize */
public int blockSize;
int bitsRequired;
long minValue;
long gcd;
long table[];

View File

@ -17,7 +17,10 @@ package org.apache.lucene.codecs.lucene46;
* 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;
@ -33,6 +36,7 @@ import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.index.SegmentWriteState;
/**
* Implements the Lucene 4.6 index format, with configurable per-field postings
@ -43,10 +47,12 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
*
* @see org.apache.lucene.codecs.lucene46 package documentation for file format details.
* @lucene.experimental
* @deprecated Only for reading old 4.6-4.8 segments
*/
// NOTE: if we make largish changes in a minor release, easier to just make Lucene46Codec or whatever
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
@Deprecated
public class Lucene46Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
@ -129,10 +135,15 @@ public class Lucene46Codec extends Codec {
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene45");
private final NormsFormat normsFormat = new Lucene42NormsFormat();
private final NormsFormat normsFormat = new Lucene42NormsFormat() {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};
@Override
public final NormsFormat normsFormat() {
public NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -0,0 +1,140 @@
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.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.Lucene42NormsFormat;
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.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 4.9 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.lucene49 package documentation for file format details.
* @lucene.experimental
*/
// NOTE: if we make largish changes in a minor release, easier to just make Lucene410Codec 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 Lucene49Codec 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 Lucene49Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Lucene49Codec.this.getDocValuesFormatForField(field);
}
};
/** Sole constructor. */
public Lucene49Codec() {
super("Lucene49");
}
@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 "Lucene49"
*/
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("Lucene49");
private final NormsFormat normsFormat = new Lucene42NormsFormat();
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -0,0 +1,469 @@
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.Closeable; // javadocs
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
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.IOUtils;
import org.apache.lucene.util.MathUtil;
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 Lucene49DocValuesFormat} */
public class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
static final int BLOCK_SIZE = 16384;
static final int ADDRESS_INTERVAL = 16;
static final Number MISSING_ORD = Long.valueOf(-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_SET_WITH_ADDRESSES = 0;
/** Single-valued sorted set values, encoded as sorted values, so no level
* of indirection: docId -> ord. */
public static final int SORTED_SET_SINGLE_VALUED_SORTED = 1;
IndexOutput data, meta;
final int maxDoc;
/** expert: Creates a new writer */
public Lucene49DocValuesConsumer(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, Lucene49DocValuesFormat.VERSION_CURRENT);
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
meta = state.directory.createOutput(metaName, state.context);
CodecUtil.writeHeader(meta, metaCodec, Lucene49DocValuesFormat.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 format;
if (uniqueValues != null
&& (delta < 0L || PackedInts.bitsRequired(uniqueValues.size() - 1) < PackedInts.bitsRequired(delta))) {
format = TABLE_COMPRESSED;
} else if (gcd != 0 && gcd != 1) {
format = GCD_COMPRESSED;
} else {
format = DELTA_COMPRESSED;
}
meta.writeVInt(field.number);
meta.writeByte(Lucene49DocValuesFormat.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 = maxDelta < 0 ? 64 : DirectWriter.bitsRequired(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);
final int bpv = delta < 0 ? 64 : DirectWriter.bitsRequired(delta);
meta.writeVInt(bpv);
final DirectWriter writer = DirectWriter.getInstance(data, count, bpv);
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()]);
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);
}
final int bitsRequired = DirectWriter.bitsRequired(uniqueValues.size() - 1);
meta.writeVInt(bitsRequired);
final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, bitsRequired);
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(Lucene49DocValuesFormat.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;
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 */
protected 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;
for (BytesRef v : values) {
minLength = Math.min(minLength, v.length);
maxLength = Math.max(maxLength, v.length);
}
if (minLength == maxLength) {
// no index needed: direct addressing by mult
addBinaryField(field, values);
} else {
// header
meta.writeVInt(field.number);
meta.writeByte(Lucene49DocValuesFormat.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);
BytesRef lastTerm = new BytesRef();
long count = 0;
for (BytesRef v : values) {
if (count % ADDRESS_INTERVAL == 0) {
termAddresses.add(data.getFilePointer() - startFP);
// force the first term in a block to be abs-encoded
lastTerm.length = 0;
}
// prefix-code
int sharedPrefix = StringHelper.bytesDifference(lastTerm, v);
data.writeVInt(sharedPrefix);
data.writeVInt(v.length - sharedPrefix);
data.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
lastTerm.copyBytes(v);
count++;
}
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.writeVInt(ADDRESS_INTERVAL);
meta.writeLong(indexStartFP);
meta.writeVInt(PackedInts.VERSION_CURRENT);
meta.writeVInt(BLOCK_SIZE);
}
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene49DocValuesFormat.SORTED);
addTermsDict(field, values);
addNumericField(field, docToOrd, false);
}
private static boolean isSingleValued(Iterable<Number> docToOrdCount) {
for (Number ordCount : docToOrdCount) {
if (ordCount.longValue() > 1) {
return false;
}
}
return true;
}
@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(Lucene49DocValuesFormat.SORTED_SET);
if (isSingleValued(docToOrdCount)) {
meta.writeVInt(SORTED_SET_SINGLE_VALUED_SORTED);
// The field is single-valued, we can encode it as SORTED
addSortedField(field, values, new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
final Iterator<Number> docToOrdCountIt = docToOrdCount.iterator();
final Iterator<Number> ordsIt = ords.iterator();
return new Iterator<Number>() {
@Override
public boolean hasNext() {
assert ordsIt.hasNext() ? docToOrdCountIt.hasNext() : true;
return docToOrdCountIt.hasNext();
}
@Override
public Number next() {
final Number ordCount = docToOrdCountIt.next();
if (ordCount.longValue() == 0) {
return MISSING_ORD;
} else {
assert ordCount.longValue() == 1;
return ordsIt.next();
}
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
});
return;
}
meta.writeVInt(SORTED_SET_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
meta.writeVInt(field.number);
meta.writeByte(Lucene49DocValuesFormat.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;
for (Number v : docToOrdCount) {
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,185 @@
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.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.9 DocValues format.
* <p>
* Encodes the four per-document value types (Numeric,Binary,Sorted,SortedSet) 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.
* </ul>
* <p>
* {@link DocValuesType#SORTED SORTED}:
* <ul>
* <li>Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-Compressed 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 Prefix-Compressed Binary,
* an ordinal 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</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>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>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 Lucene49DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */
public Lucene49DocValuesFormat() {
super("Lucene49");
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene49DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene49DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
static final String DATA_CODEC = "Lucene49DocValuesData";
static final String DATA_EXTENSION = "dvd";
static final String META_CODEC = "Lucene49ValuesMetadata";
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;
}

View File

@ -0,0 +1,908 @@
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 static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.DELTA_COMPRESSED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.GCD_COMPRESSED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.MONOTONIC_COMPRESSED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.SORTED_SET_SINGLE_VALUED_SORTED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.SORTED_SET_WITH_ADDRESSES;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.TABLE_COMPRESSED;
import java.io.Closeable; // javadocs
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.DirectReader;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
/** reader for {@link Lucene49DocValuesFormat} */
public class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
private final Map<Integer,NumericEntry> numerics;
private final Map<Integer,BinaryEntry> binaries;
private final Map<Integer,SortedSetEntry> sortedSets;
private final Map<Integer,NumericEntry> ords;
private final Map<Integer,NumericEntry> ordIndexes;
private final AtomicLong ramBytesUsed;
private final IndexInput data;
private final int maxDoc;
private final int version;
// memory-resident structures
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
/** expert: instantiates a new reader */
protected Lucene49DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
// read in the entries from the metadata file.
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
this.maxDoc = state.segmentInfo.getDocCount();
boolean success = false;
try {
version = CodecUtil.checkHeader(in, metaCodec,
Lucene49DocValuesFormat.VERSION_START,
Lucene49DocValuesFormat.VERSION_CURRENT);
numerics = new HashMap<>();
ords = new HashMap<>();
ordIndexes = new HashMap<>();
binaries = new HashMap<>();
sortedSets = new HashMap<>();
readFields(in, state.fieldInfos);
CodecUtil.checkFooter(in);
success = true;
} finally {
if (success) {
IOUtils.close(in);
} else {
IOUtils.closeWhileHandlingException(in);
}
}
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
this.data = state.directory.openInput(dataName, state.context);
success = false;
try {
final int version2 = CodecUtil.checkHeader(data, dataCodec,
Lucene49DocValuesFormat.VERSION_START,
Lucene49DocValuesFormat.VERSION_CURRENT);
if (version != version2) {
throw new CorruptIndexException("Format versions mismatch");
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this.data);
}
}
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
}
private void readSortedField(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
// sorted = binary + numeric
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.BINARY) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
NumericEntry n = readNumericEntry(meta);
ords.put(fieldNumber, n);
}
private void readSortedSetFieldWithAddresses(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
// sortedset = binary + numeric (addresses) + ordIndex
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.BINARY) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
NumericEntry n1 = readNumericEntry(meta);
ords.put(fieldNumber, n1);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
NumericEntry n2 = readNumericEntry(meta);
ordIndexes.put(fieldNumber, n2);
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
if (infos.fieldInfo(fieldNumber) == null) {
// trickier to validate more: because we re-use for norms, because we use multiple entries
// for "composite" types like sortedset, etc.
throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
}
byte type = meta.readByte();
if (type == Lucene49DocValuesFormat.NUMERIC) {
numerics.put(fieldNumber, readNumericEntry(meta));
} else if (type == Lucene49DocValuesFormat.BINARY) {
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
} else if (type == Lucene49DocValuesFormat.SORTED) {
readSortedField(fieldNumber, meta, infos);
} else if (type == Lucene49DocValuesFormat.SORTED_SET) {
SortedSetEntry ss = readSortedSetEntry(meta);
sortedSets.put(fieldNumber, ss);
if (ss.format == SORTED_SET_WITH_ADDRESSES) {
readSortedSetFieldWithAddresses(fieldNumber, meta, infos);
} else if (ss.format == SORTED_SET_SINGLE_VALUED_SORTED) {
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.SORTED) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
readSortedField(fieldNumber, meta, infos);
} else {
throw new AssertionError();
}
} else {
throw new CorruptIndexException("invalid type: " + type + ", resource=" + meta);
}
fieldNumber = meta.readVInt();
}
}
static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
NumericEntry entry = new NumericEntry();
entry.format = meta.readVInt();
entry.missingOffset = meta.readLong();
entry.offset = meta.readLong();
entry.count = meta.readVLong();
switch(entry.format) {
case GCD_COMPRESSED:
entry.minValue = meta.readLong();
entry.gcd = meta.readLong();
entry.bitsPerValue = meta.readVInt();
break;
case TABLE_COMPRESSED:
final int uniqueValues = meta.readVInt();
if (uniqueValues > 256) {
throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + meta);
}
entry.table = new long[uniqueValues];
for (int i = 0; i < uniqueValues; ++i) {
entry.table[i] = meta.readLong();
}
entry.bitsPerValue = meta.readVInt();
break;
case DELTA_COMPRESSED:
entry.minValue = meta.readLong();
entry.bitsPerValue = meta.readVInt();
break;
case MONOTONIC_COMPRESSED:
entry.packedIntsVersion = meta.readVInt();
entry.blockSize = meta.readVInt();
break;
default:
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
}
entry.endOffset = meta.readLong();
return entry;
}
static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
BinaryEntry entry = new BinaryEntry();
entry.format = meta.readVInt();
entry.missingOffset = meta.readLong();
entry.minLength = meta.readVInt();
entry.maxLength = meta.readVInt();
entry.count = meta.readVLong();
entry.offset = meta.readLong();
switch(entry.format) {
case BINARY_FIXED_UNCOMPRESSED:
break;
case BINARY_PREFIX_COMPRESSED:
entry.addressInterval = meta.readVInt();
entry.addressesOffset = meta.readLong();
entry.packedIntsVersion = meta.readVInt();
entry.blockSize = meta.readVInt();
break;
case BINARY_VARIABLE_UNCOMPRESSED:
entry.addressesOffset = meta.readLong();
entry.packedIntsVersion = meta.readVInt();
entry.blockSize = meta.readVInt();
break;
default:
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
}
return entry;
}
SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
SortedSetEntry entry = new SortedSetEntry();
entry.format = meta.readVInt();
if (entry.format != SORTED_SET_SINGLE_VALUED_SORTED && entry.format != SORTED_SET_WITH_ADDRESSES) {
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
}
return entry;
}
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericEntry entry = numerics.get(field.number);
return getNumeric(entry);
}
@Override
public long ramBytesUsed() {
return ramBytesUsed.get();
}
@Override
public void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(data);
}
LongValues getNumeric(NumericEntry entry) throws IOException {
RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
switch (entry.format) {
case DELTA_COMPRESSED:
final long delta = entry.minValue;
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
return new LongValues() {
@Override
public long get(long id) {
return delta + values.get(id);
}
};
case GCD_COMPRESSED:
final long min = entry.minValue;
final long mult = entry.gcd;
final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue);
return new LongValues() {
@Override
public long get(long id) {
return min + mult * quotientReader.get(id);
}
};
case TABLE_COMPRESSED:
final long table[] = entry.table;
final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue);
return new LongValues() {
@Override
public long get(long id) {
return table[(int) ords.get(id)];
}
};
default:
throw new AssertionError();
}
}
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryEntry bytes = binaries.get(field.number);
switch(bytes.format) {
case BINARY_FIXED_UNCOMPRESSED:
return getFixedBinary(field, bytes);
case BINARY_VARIABLE_UNCOMPRESSED:
return getVariableBinary(field, bytes);
case BINARY_PREFIX_COMPRESSED:
return getCompressedBinary(field, bytes);
default:
throw new AssertionError();
}
}
private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) {
final IndexInput data = this.data.clone();
return new LongBinaryDocValues() {
@Override
public void get(long id, BytesRef result) {
long address = bytes.offset + id * bytes.maxLength;
try {
data.seek(address);
// NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource)
// assume "they" own the bytes after calling this!
final byte[] buffer = new byte[bytes.maxLength];
data.readBytes(buffer, 0, buffer.length);
result.bytes = buffer;
result.offset = 0;
result.length = buffer.length;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
};
}
/** returns an address instance for variable-length binary values.
* @lucene.internal */
protected MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
final MonotonicBlockPackedReader addresses;
synchronized (addressInstances) {
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
addressInstances.put(field.number, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
addresses = addrInstance;
}
return addresses;
}
private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
final IndexInput data = this.data.clone();
final MonotonicBlockPackedReader addresses = getAddressInstance(data, field, bytes);
return new LongBinaryDocValues() {
@Override
public void get(long id, BytesRef result) {
long startAddress = bytes.offset + (id == 0 ? 0 : addresses.get(id-1));
long endAddress = bytes.offset + addresses.get(id);
int length = (int) (endAddress - startAddress);
try {
data.seek(startAddress);
// NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource)
// assume "they" own the bytes after calling this!
final byte[] buffer = new byte[length];
data.readBytes(buffer, 0, buffer.length);
result.bytes = buffer;
result.offset = 0;
result.length = length;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
};
}
/** returns an address instance for prefix-compressed binary values.
* @lucene.internal */
protected MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
final MonotonicBlockPackedReader addresses;
final long interval = bytes.addressInterval;
synchronized (addressInstances) {
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
final long size;
if (bytes.count % interval == 0) {
size = bytes.count / interval;
} else {
size = 1L + bytes.count / interval;
}
addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
addressInstances.put(field.number, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
addresses = addrInstance;
}
return addresses;
}
private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
final IndexInput data = this.data.clone();
final MonotonicBlockPackedReader addresses = getIntervalInstance(data, field, bytes);
return new CompressedBinaryDocValues(bytes, addresses, data);
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
final int valueCount = (int) binaries.get(field.number).count;
final BinaryDocValues binary = getBinary(field);
NumericEntry entry = ords.get(field.number);
final LongValues ordinals = getNumeric(entry);
return new SortedDocValues() {
@Override
public int getOrd(int docID) {
return (int) ordinals.get(docID);
}
@Override
public void lookupOrd(int ord, BytesRef result) {
binary.get(ord, result);
}
@Override
public int getValueCount() {
return valueCount;
}
@Override
public int lookupTerm(BytesRef key) {
if (binary instanceof CompressedBinaryDocValues) {
return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
} else {
return super.lookupTerm(key);
}
}
@Override
public TermsEnum termsEnum() {
if (binary instanceof CompressedBinaryDocValues) {
return ((CompressedBinaryDocValues)binary).getTermsEnum();
} else {
return super.termsEnum();
}
}
};
}
/** returns an address instance for sortedset ordinal lists
* @lucene.internal */
protected MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
final MonotonicBlockPackedReader ordIndex;
synchronized (ordIndexInstances) {
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
if (ordIndexInstance == null) {
data.seek(entry.offset);
ordIndexInstance = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
ordIndexInstances.put(field.number, ordIndexInstance);
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
ordIndex = ordIndexInstance;
}
return ordIndex;
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
SortedSetEntry ss = sortedSets.get(field.number);
if (ss.format == SORTED_SET_SINGLE_VALUED_SORTED) {
final SortedDocValues values = getSorted(field);
return DocValues.singleton(values);
} else if (ss.format != SORTED_SET_WITH_ADDRESSES) {
throw new AssertionError();
}
final IndexInput data = this.data.clone();
final long valueCount = binaries.get(field.number).count;
// we keep the byte[]s and list of ords on disk, these could be large
final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
final LongValues ordinals = getNumeric(ords.get(field.number));
// but the addresses to the ord stream are in RAM
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.number));
return new RandomAccessOrds() {
long startOffset;
long offset;
long endOffset;
@Override
public long nextOrd() {
if (offset == endOffset) {
return NO_MORE_ORDS;
} else {
long ord = ordinals.get(offset);
offset++;
return ord;
}
}
@Override
public void setDocument(int docID) {
startOffset = offset = (docID == 0 ? 0 : ordIndex.get(docID-1));
endOffset = ordIndex.get(docID);
}
@Override
public void lookupOrd(long ord, BytesRef result) {
binary.get(ord, result);
}
@Override
public long getValueCount() {
return valueCount;
}
@Override
public long lookupTerm(BytesRef key) {
if (binary instanceof CompressedBinaryDocValues) {
return ((CompressedBinaryDocValues)binary).lookupTerm(key);
} else {
return super.lookupTerm(key);
}
}
@Override
public TermsEnum termsEnum() {
if (binary instanceof CompressedBinaryDocValues) {
return ((CompressedBinaryDocValues)binary).getTermsEnum();
} else {
return super.termsEnum();
}
}
@Override
public long ordAt(int index) {
return ordinals.get(startOffset + index);
}
@Override
public int cardinality() {
return (int) (endOffset - startOffset);
}
};
}
private Bits getMissingBits(final long offset) throws IOException {
if (offset == -1) {
return new Bits.MatchAllBits(maxDoc);
} else {
int length = (int) ((maxDoc + 7L) >>> 3);
final RandomAccessInput in = data.randomAccessSlice(offset, length);
return new Bits() {
@Override
public boolean get(int index) {
try {
return (in.readByte(index >> 3) & (1 << (index & 7))) != 0;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public int length() {
return maxDoc;
}
};
}
}
@Override
public Bits getDocsWithField(FieldInfo field) throws IOException {
switch(field.getDocValuesType()) {
case SORTED_SET:
return DocValues.docsWithValue(getSortedSet(field), maxDoc);
case SORTED:
return DocValues.docsWithValue(getSorted(field), maxDoc);
case BINARY:
BinaryEntry be = binaries.get(field.number);
return getMissingBits(be.missingOffset);
case NUMERIC:
NumericEntry ne = numerics.get(field.number);
return getMissingBits(ne.missingOffset);
default:
throw new AssertionError();
}
}
@Override
public void close() throws IOException {
data.close();
}
/** metadata entry for a numeric docvalues field */
protected static class NumericEntry {
private NumericEntry() {}
/** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
long missingOffset;
/** offset to the actual numeric values */
public long offset;
/** end offset to the actual numeric values */
public long endOffset;
/** bits per value used to pack the numeric values */
public int bitsPerValue;
int format;
/** packed ints version used to encode these numerics */
public int packedIntsVersion;
/** count of values written */
public long count;
/** packed ints blocksize */
public int blockSize;
long minValue;
long gcd;
long table[];
}
/** metadata entry for a binary docvalues field */
protected static class BinaryEntry {
private BinaryEntry() {}
/** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
long missingOffset;
/** offset to the actual binary values */
long offset;
int format;
/** count of values written */
public long count;
int minLength;
int maxLength;
/** offset to the addressing data that maps a value to its slice of the byte[] */
public long addressesOffset;
/** interval of shared prefix chunks (when using prefix-compressed binary) */
public long addressInterval;
/** packed ints version used to encode addressing information */
public int packedIntsVersion;
/** packed ints blocksize */
public int blockSize;
}
/** metadata entry for a sorted-set docvalues field */
protected static class SortedSetEntry {
private SortedSetEntry() {}
int format;
}
// internally we compose complex dv (sorted/sortedset) from other ones
static abstract class LongBinaryDocValues extends BinaryDocValues {
@Override
public final void get(int docID, BytesRef result) {
get((long)docID, result);
}
abstract void get(long id, BytesRef Result);
}
// in the compressed case, we add a few additional operations for
// more efficient reverse lookup and enumeration
static class CompressedBinaryDocValues extends LongBinaryDocValues {
final BinaryEntry bytes;
final long interval;
final long numValues;
final long numIndexValues;
final MonotonicBlockPackedReader addresses;
final IndexInput data;
final TermsEnum termsEnum;
public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, IndexInput data) throws IOException {
this.bytes = bytes;
this.interval = bytes.addressInterval;
this.addresses = addresses;
this.data = data;
this.numValues = bytes.count;
this.numIndexValues = addresses.size();
this.termsEnum = getTermsEnum(data);
}
@Override
public void get(long id, BytesRef result) {
try {
termsEnum.seekExact(id);
BytesRef term = termsEnum.term();
result.bytes = term.bytes;
result.offset = term.offset;
result.length = term.length;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
long lookupTerm(BytesRef key) {
try {
SeekStatus status = termsEnum.seekCeil(key);
if (status == SeekStatus.END) {
return -numValues-1;
} else if (status == SeekStatus.FOUND) {
return termsEnum.ord();
} else {
return -termsEnum.ord()-1;
}
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
}
TermsEnum getTermsEnum() {
try {
return getTermsEnum(data.clone());
} catch (IOException e) {
throw new RuntimeException(e);
}
}
private TermsEnum getTermsEnum(final IndexInput input) throws IOException {
input.seek(bytes.offset);
return new TermsEnum() {
private long currentOrd = -1;
// TODO: maxLength is negative when all terms are merged away...
private final BytesRef termBuffer = new BytesRef(bytes.maxLength < 0 ? 0 : bytes.maxLength);
private final BytesRef term = new BytesRef(); // TODO: paranoia?
@Override
public BytesRef next() throws IOException {
if (doNext() == null) {
return null;
} else {
setTerm();
return term;
}
}
private BytesRef doNext() throws IOException {
if (++currentOrd >= numValues) {
return null;
} else {
int start = input.readVInt();
int suffix = input.readVInt();
input.readBytes(termBuffer.bytes, start, suffix);
termBuffer.length = start + suffix;
return termBuffer;
}
}
@Override
public SeekStatus seekCeil(BytesRef text) throws IOException {
// binary-search just the index values to find the block,
// then scan within the block
long low = 0;
long high = numIndexValues-1;
while (low <= high) {
long mid = (low + high) >>> 1;
doSeek(mid * interval);
int cmp = termBuffer.compareTo(text);
if (cmp < 0) {
low = mid + 1;
} else if (cmp > 0) {
high = mid - 1;
} else {
// we got lucky, found an indexed term
setTerm();
return SeekStatus.FOUND;
}
}
if (numIndexValues == 0) {
return SeekStatus.END;
}
// block before insertion point
long block = low-1;
doSeek(block < 0 ? -1 : block * interval);
while (doNext() != null) {
int cmp = termBuffer.compareTo(text);
if (cmp == 0) {
setTerm();
return SeekStatus.FOUND;
} else if (cmp > 0) {
setTerm();
return SeekStatus.NOT_FOUND;
}
}
return SeekStatus.END;
}
@Override
public void seekExact(long ord) throws IOException {
doSeek(ord);
setTerm();
}
private void doSeek(long ord) throws IOException {
long block = ord / interval;
if (ord >= currentOrd && block == currentOrd / interval) {
// seek within current block
} else {
// position before start of block
currentOrd = ord - ord % interval - 1;
input.seek(bytes.offset + addresses.get(block));
}
while (currentOrd < ord) {
doNext();
}
}
private void setTerm() {
// TODO: is there a cleaner way
term.bytes = new byte[termBuffer.length];
term.offset = 0;
term.copyBytes(termBuffer);
}
@Override
public BytesRef term() throws IOException {
return term;
}
@Override
public long ord() throws IOException {
return currentOrd;
}
@Override
public int docFreq() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public long totalTermFreq() throws IOException {
return -1;
}
@Override
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}
};
}
}
}

View File

@ -0,0 +1,401 @@
<!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.9 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.lucene42.Lucene42NormsFormat 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.lucene49.Lucene49DocValuesFormat 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.lucene42.Lucene42NormsFormat 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.lucene49.Lucene49DocValuesFormat 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>
</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

@ -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.lucene46.Lucene46Codec} and override
{@link org.apache.lucene.codecs.lucene46.Lucene46Codec#getPostingsFormatForField(String)} to return your custom
{@link org.apache.lucene.codecs.lucene49.Lucene49Codec} and override
{@link org.apache.lucene.codecs.lucene49.Lucene49Codec#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.lucene46.Lucene46Codec#getDocValuesFormatForField(String)}.
a look at {@link org.apache.lucene.codecs.lucene49.Lucene49Codec#getDocValuesFormatForField(String)}.
</p>
</body>
</html>

View File

@ -36,7 +36,7 @@ import org.apache.lucene.util.WeakIdentityMap;
* For efficiency, this class requires that the buffers
* are a power-of-two (<code>chunkSizePower</code>).
*/
abstract class ByteBufferIndexInput extends IndexInput {
abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessInput {
protected final BufferCleaner cleaner;
protected final long length;
protected final long chunkSizeMask;
@ -178,6 +178,76 @@ abstract class ByteBufferIndexInput extends IndexInput {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public byte readByte(long pos) throws IOException {
try {
final int bi = (int) (pos >> chunkSizePower);
return buffers[bi].get((int) (pos & chunkSizeMask));
} catch (IndexOutOfBoundsException ioobe) {
throw new EOFException("seek past EOF: " + this);
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
// used only by random access methods to handle reads across boundaries
private void setPos(long pos, int bi) throws IOException {
try {
final ByteBuffer b = buffers[bi];
b.position((int) (pos & chunkSizeMask));
this.curBufIndex = bi;
this.curBuf = b;
} catch (ArrayIndexOutOfBoundsException aioobe) {
throw new EOFException("seek past EOF: " + this);
} catch (IllegalArgumentException iae) {
throw new EOFException("seek past EOF: " + this);
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public short readShort(long pos) throws IOException {
final int bi = (int) (pos >> chunkSizePower);
try {
return buffers[bi].getShort((int) (pos & chunkSizeMask));
} catch (IndexOutOfBoundsException ioobe) {
// either its a boundary, or read past EOF, fall back:
setPos(pos, bi);
return readShort();
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public int readInt(long pos) throws IOException {
final int bi = (int) (pos >> chunkSizePower);
try {
return buffers[bi].getInt((int) (pos & chunkSizeMask));
} catch (IndexOutOfBoundsException ioobe) {
// either its a boundary, or read past EOF, fall back:
setPos(pos, bi);
return readInt();
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public long readLong(long pos) throws IOException {
final int bi = (int) (pos >> chunkSizePower);
try {
return buffers[bi].getLong((int) (pos & chunkSizeMask));
} catch (IndexOutOfBoundsException ioobe) {
// either its a boundary, or read past EOF, fall back:
setPos(pos, bi);
return readLong();
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public final long length() {
@ -208,6 +278,12 @@ abstract class ByteBufferIndexInput extends IndexInput {
return buildSlice(sliceDescription, offset, length);
}
@Override
public RandomAccessInput randomAccessSlice(long offset, long length) throws IOException {
// note: technically we could even avoid the clone...
return slice(null, offset, length);
}
/** Builds the actual sliced IndexInput (may apply extra offset in subclasses). **/
protected ByteBufferIndexInput buildSlice(String sliceDescription, long offset, long length) {
if (buffers == null) {
@ -373,6 +449,66 @@ abstract class ByteBufferIndexInput extends IndexInput {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public byte readByte(long pos) throws IOException {
try {
return curBuf.get((int) pos);
} catch (IllegalArgumentException e) {
if (pos < 0) {
throw new IllegalArgumentException("Seeking to negative position: " + this, e);
} else {
throw new EOFException("seek past EOF: " + this);
}
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public short readShort(long pos) throws IOException {
try {
return curBuf.getShort((int) pos);
} catch (IllegalArgumentException e) {
if (pos < 0) {
throw new IllegalArgumentException("Seeking to negative position: " + this, e);
} else {
throw new EOFException("seek past EOF: " + this);
}
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public int readInt(long pos) throws IOException {
try {
return curBuf.getInt((int) pos);
} catch (IllegalArgumentException e) {
if (pos < 0) {
throw new IllegalArgumentException("Seeking to negative position: " + this, e);
} else {
throw new EOFException("seek past EOF: " + this);
}
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
@Override
public long readLong(long pos) throws IOException {
try {
return curBuf.getLong((int) pos);
} catch (IllegalArgumentException e) {
if (pos < 0) {
throw new IllegalArgumentException("Seeking to negative position: " + this, e);
} else {
throw new EOFException("seek past EOF: " + this);
}
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
}
}
/** This class adds offset support to ByteBufferIndexInput, which is needed for slices. */
@ -404,6 +540,26 @@ abstract class ByteBufferIndexInput extends IndexInput {
return super.getFilePointer() - offset;
}
@Override
public byte readByte(long pos) throws IOException {
return super.readByte(pos + offset);
}
@Override
public short readShort(long pos) throws IOException {
return super.readShort(pos + offset);
}
@Override
public int readInt(long pos) throws IOException {
return super.readInt(pos + offset);
}
@Override
public long readLong(long pos) throws IOException {
return super.readLong(pos + offset);
}
@Override
protected ByteBufferIndexInput buildSlice(String sliceDescription, long ofs, long length) {
return super.buildSlice(sliceDescription, this.offset + ofs, length);

View File

@ -88,4 +88,39 @@ public abstract class IndexInput extends DataInput implements Cloneable,Closeabl
* The slice is seeked to the beginning.
*/
public abstract IndexInput slice(String sliceDescription, long offset, long length) throws IOException;
/**
* Creates a random-access slice of this index input, with the given offset and length.
* <p>
* The default implementation calls {@link #slice}, and implements absolute reads as
* seek+read.
*/
public RandomAccessInput randomAccessSlice(long offset, long length) throws IOException {
final IndexInput slice = slice("randomaccess", offset, length);
return new RandomAccessInput() {
@Override
public byte readByte(long pos) throws IOException {
slice.seek(pos);
return slice.readByte();
}
@Override
public short readShort(long pos) throws IOException {
slice.seek(pos);
return slice.readShort();
}
@Override
public int readInt(long pos) throws IOException {
slice.seek(pos);
return slice.readInt();
}
@Override
public long readLong(long pos) throws IOException {
slice.seek(pos);
return slice.readLong();
}
};
}
}

View File

@ -0,0 +1,49 @@
package org.apache.lucene.store;
/*
* 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;
/**
* Random Access Index API.
* Unlike {@link IndexInput}, this has no concept of file position, all reads
* are absolute. However, like IndexInput, it is only intended for use by a single thread.
*/
public interface RandomAccessInput {
/**
* Reads a byte at the given position in the file
* @see DataInput#readByte
*/
public byte readByte(long pos) throws IOException;
/**
* Reads a short at the given position in the file
* @see DataInput#readShort
*/
public short readShort(long pos) throws IOException;
/**
* Reads an integer at the given position in the file
* @see DataInput#readInt
*/
public int readInt(long pos) throws IOException;
/**
* Reads a long at the given position in the file
* @see DataInput#readLong
*/
public long readLong(long pos) throws IOException;
}

View File

@ -58,16 +58,13 @@ import org.apache.lucene.store.DataOutput;
* @lucene.internal
*/
public final class BlockPackedWriter extends AbstractBlockPackedWriter {
final float acceptableOverheadRatio;
/**
* Sole constructor.
* @param blockSize the number of values of a single block, must be a power of 2
* @param acceptableOverheadRatio an acceptable overhead ratio per value
*/
public BlockPackedWriter(DataOutput out, int blockSize, float acceptableOverheadRatio) {
public BlockPackedWriter(DataOutput out, int blockSize) {
super(out, blockSize);
this.acceptableOverheadRatio = acceptableOverheadRatio;
}
protected void flush() throws IOException {
@ -80,7 +77,6 @@ public final class BlockPackedWriter extends AbstractBlockPackedWriter {
final long delta = max - min;
int bitsRequired = delta < 0 ? 64 : delta == 0L ? 0 : PackedInts.bitsRequired(delta);
bitsRequired = PackedInts.fastestDirectBits(bitsRequired, acceptableOverheadRatio);
if (bitsRequired == 64) {
// no need to delta-encode
min = 0L;

View File

@ -22,6 +22,7 @@ import org.apache.lucene.store.IndexInput;
import java.io.IOException;
/* Reads directly from disk on each get */
// just for back compat, use DirectReader/DirectWriter for more efficient impl
class DirectPackedReader extends PackedInts.ReaderImpl {
final IndexInput in;
final long startPointer;
@ -98,258 +99,4 @@ class DirectPackedReader extends PackedInts.ReaderImpl {
public long ramBytesUsed() {
return 0;
}
static class DirectPackedReader1 extends DirectPackedReader {
DirectPackedReader1(int valueCount, IndexInput in) {
super(1, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index >>> 3));
int shift = 7 - (index & 7);
return (in.readByte() >>> shift) & 0x1;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader2 extends DirectPackedReader {
DirectPackedReader2(int valueCount, IndexInput in) {
super(2, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index >>> 2));
int shift = (3 - (index & 3)) << 1;
return (in.readByte() >>> shift) & 0x3;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader4 extends DirectPackedReader {
DirectPackedReader4(int valueCount, IndexInput in) {
super(4, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index >>> 1));
int shift = ((index + 1) & 1) << 2;
return (in.readByte() >>> shift) & 0xF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader8 extends DirectPackedReader {
DirectPackedReader8(int valueCount, IndexInput in) {
super(8, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + index);
return in.readByte() & 0xFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader12 extends DirectPackedReader {
DirectPackedReader12(int valueCount, IndexInput in) {
super(12, valueCount, in);
}
@Override
public long get(int index) {
try {
long offset = (index * 12L) >>> 3;
in.seek(startPointer + offset);
int shift = ((index + 1) & 1) << 2;
return (in.readShort() >>> shift) & 0xFFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader16 extends DirectPackedReader {
DirectPackedReader16(int valueCount, IndexInput in) {
super(16, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index<<1));
return in.readShort() & 0xFFFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader20 extends DirectPackedReader {
DirectPackedReader20(int valueCount, IndexInput in) {
super(20, valueCount, in);
}
@Override
public long get(int index) {
try {
long offset = (index * 20L) >>> 3;
in.seek(startPointer + offset);
int v = in.readShort() << 8 | (in.readByte() & 0xFF);
int shift = ((index + 1) & 1) << 2;
return (v >>> shift) & 0xFFFFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader24 extends DirectPackedReader {
DirectPackedReader24(int valueCount, IndexInput in) {
super(24, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index*3));
return (in.readShort() & 0xFFFF) << 8 | (in.readByte() & 0xFF);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader28 extends DirectPackedReader {
DirectPackedReader28(int valueCount, IndexInput in) {
super(28, valueCount, in);
}
@Override
public long get(int index) {
try {
long offset = (index * 28L) >>> 3;
in.seek(startPointer + offset);
int shift = ((index + 1) & 1) << 2;
return (in.readInt() >>> shift) & 0xFFFFFFFL;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader32 extends DirectPackedReader {
DirectPackedReader32(int valueCount, IndexInput in) {
super(32, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index<<2));
return in.readInt() & 0xFFFFFFFFL;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader40 extends DirectPackedReader {
DirectPackedReader40(int valueCount, IndexInput in) {
super(40, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index*5));
return (in.readInt() & 0xFFFFFFFFL) << 8 | (in.readByte() & 0xFF);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader48 extends DirectPackedReader {
DirectPackedReader48(int valueCount, IndexInput in) {
super(48, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index*6));
return (in.readInt() & 0xFFFFFFFFL) << 16 | (in.readShort() & 0xFFFF);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader56 extends DirectPackedReader {
DirectPackedReader56(int valueCount, IndexInput in) {
super(56, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index*7));
return (in.readInt() & 0xFFFFFFFFL) << 24 | (in.readShort() & 0xFFFF) << 8 | (in.readByte() & 0xFF);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static class DirectPackedReader64 extends DirectPackedReader {
DirectPackedReader64(int valueCount, IndexInput in) {
super(64, valueCount, in);
}
@Override
public long get(int index) {
try {
in.seek(startPointer + (index<<3));
return in.readLong();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static DirectPackedReader getInstance(int bitsPerValue, int valueCount, IndexInput in) {
switch(bitsPerValue) {
case 1: return new DirectPackedReader1(valueCount, in);
case 2: return new DirectPackedReader2(valueCount, in);
case 4: return new DirectPackedReader4(valueCount, in);
case 8: return new DirectPackedReader8(valueCount, in);
case 12: return new DirectPackedReader12(valueCount, in);
case 16: return new DirectPackedReader16(valueCount, in);
case 20: return new DirectPackedReader20(valueCount, in);
case 24: return new DirectPackedReader24(valueCount, in);
case 28: return new DirectPackedReader28(valueCount, in);
case 32: return new DirectPackedReader32(valueCount, in);
case 40: return new DirectPackedReader40(valueCount, in);
case 48: return new DirectPackedReader48(valueCount, in);
case 56: return new DirectPackedReader56(valueCount, in);
case 64: return new DirectPackedReader64(valueCount, in);
default: return new DirectPackedReader(bitsPerValue, valueCount, in);
}
}
}

View File

@ -0,0 +1,313 @@
package org.apache.lucene.util.packed;
/*
* 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.store.RandomAccessInput;
import org.apache.lucene.util.LongValues;
/**
* Retrieves an instance previously written by {@link DirectWriter}
* <p>
* Example usage:
* <pre class="prettyprint">
* int bitsPerValue = 100;
* IndexInput in = dir.openInput("packed", IOContext.DEFAULT);
* LongValues values = DirectReader.getInstance(in.randomAccessSlice(start, end), bitsPerValue);
* for (int i = 0; i < numValues; i++) {
* long value = values.get(i);
* }
* </pre>
* @see DirectWriter
*/
public class DirectReader {
/**
* Retrieves an instance from the specified slice written decoding
* {@code bitsPerValue} for each value
*/
public static LongValues getInstance(RandomAccessInput slice, int bitsPerValue) {
switch (bitsPerValue) {
case 1: return new DirectPackedReader1(slice);
case 2: return new DirectPackedReader2(slice);
case 4: return new DirectPackedReader4(slice);
case 8: return new DirectPackedReader8(slice);
case 12: return new DirectPackedReader12(slice);
case 16: return new DirectPackedReader16(slice);
case 20: return new DirectPackedReader20(slice);
case 24: return new DirectPackedReader24(slice);
case 28: return new DirectPackedReader28(slice);
case 32: return new DirectPackedReader32(slice);
case 40: return new DirectPackedReader40(slice);
case 48: return new DirectPackedReader48(slice);
case 56: return new DirectPackedReader56(slice);
case 64: return new DirectPackedReader64(slice);
default: throw new IllegalArgumentException("unsupported bitsPerValue: " + bitsPerValue);
}
}
static final class DirectPackedReader1 extends LongValues {
final RandomAccessInput in;
DirectPackedReader1(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
int shift = 7 - (int) (index & 7);
return (in.readByte(index >>> 3) >>> shift) & 0x1;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader2 extends LongValues {
final RandomAccessInput in;
DirectPackedReader2(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
int shift = (3 - (int)(index & 3)) << 1;
return (in.readByte(index >>> 2) >>> shift) & 0x3;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader4 extends LongValues {
final RandomAccessInput in;
DirectPackedReader4(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
int shift = (int) ((index + 1) & 1) << 2;
return (in.readByte(index >>> 1) >>> shift) & 0xF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader8 extends LongValues {
final RandomAccessInput in;
DirectPackedReader8(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
return in.readByte(index) & 0xFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader12 extends LongValues {
final RandomAccessInput in;
DirectPackedReader12(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
long offset = (index * 12) >>> 3;
int shift = (int) ((index + 1) & 1) << 2;
return (in.readShort(offset) >>> shift) & 0xFFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader16 extends LongValues {
final RandomAccessInput in;
DirectPackedReader16(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
return in.readShort(index << 1) & 0xFFFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader20 extends LongValues {
final RandomAccessInput in;
DirectPackedReader20(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
long offset = (index * 20) >>> 3;
// TODO: clean this up...
int v = in.readInt(offset) >>> 8;
int shift = (int) ((index + 1) & 1) << 2;
return (v >>> shift) & 0xFFFFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader24 extends LongValues {
final RandomAccessInput in;
DirectPackedReader24(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
return in.readInt(index * 3) >>> 8;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader28 extends LongValues {
final RandomAccessInput in;
DirectPackedReader28(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
long offset = (index * 28) >>> 3;
int shift = (int) ((index + 1) & 1) << 2;
return (in.readInt(offset) >>> shift) & 0xFFFFFFFL;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader32 extends LongValues {
final RandomAccessInput in;
DirectPackedReader32(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
return in.readInt(index << 2) & 0xFFFFFFFFL;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader40 extends LongValues {
final RandomAccessInput in;
DirectPackedReader40(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
return in.readLong(index * 5) >>> 24;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader48 extends LongValues {
final RandomAccessInput in;
DirectPackedReader48(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
return in.readLong(index * 6) >>> 16;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader56 extends LongValues {
final RandomAccessInput in;
DirectPackedReader56(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
return in.readLong(index * 7) >>> 8;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
static final class DirectPackedReader64 extends LongValues {
final RandomAccessInput in;
DirectPackedReader64(RandomAccessInput in) {
this.in = in;
}
@Override
public long get(long index) {
try {
return in.readLong(index << 3);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
}

View File

@ -0,0 +1,136 @@
package org.apache.lucene.util.packed;
/*
* 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.EOFException;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.IndexOutput;
/**
* Class for writing packed integers to be directly read from Directory.
* Integers can be read on-the-fly via {@link DirectReader}.
* <p>
* Unlike PackedInts, it optimizes for read i/o operations and supports > 2B values.
* Example usage:
* <pre class="prettyprint">
* int bitsPerValue = DirectWriter.bitsRequired(100); // values up to and including 100
* IndexOutput output = dir.createOutput("packed", IOContext.DEFAULT);
* DirectWriter writer = DirectWriter.getInstance(output, numberOfValues, bitsPerValue);
* for (int i = 0; i < numberOfValues; i++) {
* writer.add(value);
* }
* writer.finish();
* output.close();
* </pre>
* @see DirectReader
*/
public final class DirectWriter {
final int bitsPerValue;
final long numValues;
final IndexOutput output;
long count;
boolean finished;
// for now, just use the existing writer under the hood
int off;
final byte[] nextBlocks;
final long[] nextValues;
final BulkOperation encoder;
final int iterations;
DirectWriter(IndexOutput output, long numValues, int bitsPerValue) {
this.output = output;
this.numValues = numValues;
this.bitsPerValue = bitsPerValue;
encoder = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
iterations = encoder.computeIterations((int) Math.min(numValues, Integer.MAX_VALUE), PackedInts.DEFAULT_BUFFER_SIZE);
nextBlocks = new byte[iterations * encoder.byteBlockCount()];
nextValues = new long[iterations * encoder.byteValueCount()];
}
/** Adds a value to this writer */
public void add(long l) throws IOException {
assert bitsPerValue == 64 || (l >= 0 && l <= PackedInts.maxValue(bitsPerValue)) : bitsPerValue;
assert !finished;
if (count >= numValues) {
throw new EOFException("Writing past end of stream");
}
nextValues[off++] = l;
if (off == nextValues.length) {
flush();
}
count++;
}
private void flush() throws IOException {
encoder.encode(nextValues, 0, nextBlocks, 0, iterations);
final int blockCount = (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
output.writeBytes(nextBlocks, blockCount);
Arrays.fill(nextValues, 0L);
off = 0;
}
/** finishes writing */
public void finish() throws IOException {
if (count != numValues) {
throw new IllegalStateException("Wrong number of values added, expected: " + numValues + ", got: " + count);
}
assert !finished;
flush();
// pad for fast io: we actually only need this for certain BPV, but its just 3 bytes...
for (int i = 0; i < 3; i++) {
output.writeByte((byte) 0);
}
finished = true;
}
/** Returns an instance suitable for encoding {@code numValues} using {@code bitsPerValue} */
public static DirectWriter getInstance(IndexOutput output, long numValues, int bitsPerValue) {
if (Arrays.binarySearch(SUPPORTED_BITS_PER_VALUE, bitsPerValue) < 0) {
throw new IllegalArgumentException("Unsupported bitsPerValue " + bitsPerValue + ". Did you use bitsRequired?");
}
return new DirectWriter(output, numValues, bitsPerValue);
}
/**
* Returns how many bits are required to hold values up
* to and including maxValue
*
* @param maxValue the maximum value that should be representable.
* @return the amount of bits needed to represent values from 0 to maxValue.
*/
public static int bitsRequired(long maxValue) {
if (maxValue < 0) {
throw new IllegalArgumentException("maxValue must be non-negative (got: " + maxValue + ")");
}
int bitsRequired = Math.max(1, 64 - Long.numberOfLeadingZeros(maxValue));
int index = Arrays.binarySearch(SUPPORTED_BITS_PER_VALUE, bitsRequired);
if (index < 0) {
return SUPPORTED_BITS_PER_VALUE[-index-1];
} else {
return bitsRequired;
}
}
final static int SUPPORTED_BITS_PER_VALUE[] = new int[] {
1, 2, 4, 8, 12, 16, 20, 24, 28, 32, 40, 48, 56, 64
};
}

View File

@ -282,39 +282,6 @@ public class PackedInts {
return new FormatAndBits(format, actualBitsPerValue);
}
/**
* Try to find the number of bits per value that would
* read from disk the fastest reader whose overhead is less than
* <code>acceptableOverheadRatio</code>.
* </p><p>
* The <code>acceptableOverheadRatio</code> parameter makes sense for
* random-access {@link Reader}s. In case you only plan to perform
* sequential access on this stream later on, you should probably use
* {@link PackedInts#COMPACT}.
* </p><p>
*/
public static int fastestDirectBits(int bitsPerValue, float acceptableOverheadRatio) {
acceptableOverheadRatio = Math.max(COMPACT, acceptableOverheadRatio);
acceptableOverheadRatio = Math.min(FASTEST, acceptableOverheadRatio);
float acceptableOverheadPerValue = acceptableOverheadRatio * bitsPerValue; // in bits
int maxBitsPerValue = bitsPerValue + (int) acceptableOverheadPerValue;
// first see if we can upgrade to byte
int byteAlign = (bitsPerValue + 7) & 0xF8;
if (byteAlign <= maxBitsPerValue) {
return byteAlign;
}
// otherwise try to upgrade to a nibble boundary (for numbers < 32)
int nibbleAlign = (bitsPerValue + 3) & 0xFC;
if (bitsPerValue < 32 && nibbleAlign <= maxBitsPerValue) {
return nibbleAlign;
}
return bitsPerValue;
}
/**
* A decoder for packed integers.
@ -997,7 +964,7 @@ public class PackedInts {
}
};
} else {
return DirectPackedReader.getInstance(bitsPerValue, valueCount, in);
return new DirectPackedReader(bitsPerValue, valueCount, in);
}
case PACKED_SINGLE_BLOCK:
return new DirectPacked64SingleBlockReader(bitsPerValue, valueCount, in);

View File

@ -17,4 +17,5 @@ org.apache.lucene.codecs.lucene40.Lucene40Codec
org.apache.lucene.codecs.lucene41.Lucene41Codec
org.apache.lucene.codecs.lucene42.Lucene42Codec
org.apache.lucene.codecs.lucene45.Lucene45Codec
org.apache.lucene.codecs.lucene46.Lucene46Codec
org.apache.lucene.codecs.lucene46.Lucene46Codec
org.apache.lucene.codecs.lucene49.Lucene49Codec

View File

@ -14,4 +14,5 @@
# limitations under the License.
org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat
org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat
org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat
org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat

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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
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 Lucene46Codec {
private static final class CustomPerFieldCodec extends Lucene49Codec {
private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");

View File

@ -0,0 +1,34 @@
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.Codec;
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
import org.apache.lucene.util.TestUtil;
/**
* Tests Lucene49DocValuesFormat
*/
public class TestLucene49DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene49DocValuesFormat());
@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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -80,9 +80,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(TEST_VERSION_CURRENT, analyzer);
final DocValuesFormat fast = DocValuesFormat.forName("Lucene45");
final DocValuesFormat fast = DocValuesFormat.forName("Lucene49");
final DocValuesFormat slow = DocValuesFormat.forName("SimpleText");
iwc.setCodec(new Lucene46Codec() {
iwc.setCodec(new Lucene49Codec() {
@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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
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 Lucene46Codec {
public static class MockCodec extends Lucene49Codec {
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 Lucene46Codec {
public static class MockCodec2 extends Lucene49Codec {
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 Lucene46Codec() {
Codec codec = new Lucene49Codec() {
@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 Lucene46Codec() {
Codec codec = new Lucene49Codec() {
@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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -1073,7 +1073,7 @@ public class TestAddIndexes extends LuceneTestCase {
aux2.close();
}
private static final class CustomPerFieldCodec extends Lucene46Codec {
private static final class CustomPerFieldCodec extends Lucene49Codec {
private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final PostingsFormat memoryFormat = PostingsFormat.forName("Memory");
@ -1124,7 +1124,7 @@ public class TestAddIndexes extends LuceneTestCase {
private static final class UnRegisteredCodec extends FilterCodec {
public UnRegisteredCodec() {
super("NotRegistered", new Lucene46Codec());
super("NotRegistered", new Lucene49Codec());
}
}

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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
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(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setCodec(new Lucene46Codec());
conf.setCodec(new Lucene49Codec());
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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
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(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setCodec(new Lucene46Codec());
conf.setCodec(new Lucene49Codec());
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
Document doc = new Document();
// these fields should sometimes get term vectors, etc

View File

@ -78,7 +78,7 @@ import org.junit.BeforeClass;
// 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"})
@SuppressCodecs({"Lucene40", "Lucene41", "Lucene42", "Lucene45", "Lucene46"})
public class TestBackwardsCompatibility extends LuceneTestCase {
// Uncomment these cases & run them on an older Lucene version,

View File

@ -17,7 +17,8 @@ import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
import org.apache.lucene.codecs.lucene45.Lucene45RWCodec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
@ -500,10 +501,10 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
public void testDifferentDVFormatPerField() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setCodec(new Lucene46Codec() {
conf.setCodec(new Lucene49Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new Lucene45DocValuesFormat();
return new Lucene49DocValuesFormat();
}
});
IndexWriter writer = new IndexWriter(dir, conf);
@ -1077,10 +1078,10 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setMergePolicy(NoMergePolicy.INSTANCE); // disable merges to simplify test assertions.
conf.setCodec(new Lucene46Codec() {
conf.setCodec(new Lucene49Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new Lucene45DocValuesFormat();
return new Lucene49DocValuesFormat();
}
});
IndexWriter writer = new IndexWriter(dir, conf);
@ -1094,7 +1095,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
// change format
conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setMergePolicy(NoMergePolicy.INSTANCE); // disable merges to simplify test assertions.
conf.setCodec(new Lucene46Codec() {
conf.setCodec(new Lucene49Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new AssertingDocValuesFormat();

View File

@ -628,7 +628,7 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
MockDirectoryWrapper dir = newMockDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(Codec.forName("Lucene46"));
iwc.setCodec(Codec.forName("Lucene49"));
IndexWriter w = new IndexWriter(dir, iwc);
Document doc = new Document();
doc.add(newStringField("id", "id", Field.Store.NO));

View File

@ -49,7 +49,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
public void setUp() throws Exception {
super.setUp();
// for now its SimpleText vs Lucene46(random postings format)
// for now its SimpleText vs Lucene49(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

@ -276,7 +276,7 @@ public class TestIndexWriterThreadsToSegments extends LuceneTestCase {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setRAMBufferSizeMB(.2);
Codec codec = Codec.forName("Lucene46");
Codec codec = Codec.forName("Lucene49");
iwc.setCodec(codec);
iwc.setMergePolicy(NoMergePolicy.INSTANCE);
final IndexWriter w = new IndexWriter(dir, iwc);

View File

@ -15,9 +15,9 @@ import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
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.Lucene45DocValuesFormat;
import org.apache.lucene.codecs.lucene45.Lucene45RWCodec;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
@ -480,10 +480,10 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
public void testDifferentDVFormatPerField() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setCodec(new Lucene46Codec() {
conf.setCodec(new Lucene49Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new Lucene45DocValuesFormat();
return new Lucene49DocValuesFormat();
}
});
IndexWriter writer = new IndexWriter(dir, conf);
@ -1059,10 +1059,10 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setMergePolicy(NoMergePolicy.INSTANCE); // disable merges to simplify test assertions.
conf.setCodec(new Lucene46Codec() {
conf.setCodec(new Lucene49Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new Lucene45DocValuesFormat();
return new Lucene49DocValuesFormat();
}
});
IndexWriter writer = new IndexWriter(dir, conf);
@ -1076,7 +1076,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
// change format
conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setMergePolicy(NoMergePolicy.INSTANCE); // disable merges to simplify test assertions.
conf.setCodec(new Lucene46Codec() {
conf.setCodec(new Lucene49Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return new AssertingDocValuesFormat();

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("Lucene46"));
iwc.setCodec(Codec.forName("Lucene49"));
iwc.setMergeScheduler(new SerialMergeScheduler());
iwc.setMaxBufferedDocs(100);
iwc.setRAMBufferSizeMB(-1);

View File

@ -26,8 +26,8 @@ import org.apache.lucene.codecs.Codec;
public class TestNamedSPILoader extends LuceneTestCase {
public void testLookup() {
Codec codec = Codec.forName("Lucene46");
assertEquals("Lucene46", codec.getName());
Codec codec = Codec.forName("Lucene49");
assertEquals("Lucene49", 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("Lucene46"));
assertTrue(codecs.contains("Lucene49"));
}
}

View File

@ -0,0 +1,134 @@
package org.apache.lucene.util.packed;
/*
* 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.Random;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MMapDirectory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.packed.DirectReader;
import org.apache.lucene.util.packed.DirectWriter;
public class TestDirectPacked extends LuceneTestCase {
/** simple encode/decode */
public void testSimple() throws Exception {
Directory dir = newDirectory();
int bitsPerValue = DirectWriter.bitsRequired(2);
IndexOutput output = dir.createOutput("foo", IOContext.DEFAULT);
DirectWriter writer = DirectWriter.getInstance(output, 5, bitsPerValue);
writer.add(1);
writer.add(0);
writer.add(2);
writer.add(1);
writer.add(2);
writer.finish();
output.close();
IndexInput input = dir.openInput("foo", IOContext.DEFAULT);
NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsPerValue);
assertEquals(1, reader.get(0));
assertEquals(0, reader.get(1));
assertEquals(2, reader.get(2));
assertEquals(1, reader.get(3));
assertEquals(2, reader.get(4));
input.close();
dir.close();
}
/** test exception is delivered if you add the wrong number of values */
public void testNotEnoughValues() throws Exception {
Directory dir = newDirectory();
int bitsPerValue = DirectWriter.bitsRequired(2);
IndexOutput output = dir.createOutput("foo", IOContext.DEFAULT);
DirectWriter writer = DirectWriter.getInstance(output, 5, bitsPerValue);
writer.add(1);
writer.add(0);
writer.add(2);
writer.add(1);
try {
writer.finish();
fail("didn't get expected exception");
} catch (IllegalStateException expected) {
assertTrue(expected.getMessage().startsWith("Wrong number of values added"));
}
output.close();
dir.close();
}
public void testRandom() throws Exception {
Directory dir = newDirectory();
for (int bpv = 1; bpv <= 64; bpv++) {
doTestBpv(dir, bpv);
}
dir.close();
}
private void doTestBpv(Directory directory, int bpv) throws Exception {
MyRandom random = new MyRandom(random().nextLong());
for (int i = 0; i < 100; i++) {
long original[] = randomLongs(random, bpv);
int bitsRequired = bpv == 64 ? 64 : DirectWriter.bitsRequired(1L<<(bpv-1));
String name = "bpv" + bpv + "_" + i;
IndexOutput output = directory.createOutput(name, IOContext.DEFAULT);
DirectWriter writer = DirectWriter.getInstance(output, original.length, bitsRequired);
for (int j = 0; j < original.length; j++) {
writer.add(original[j]);
}
writer.finish();
output.close();
IndexInput input = directory.openInput(name, IOContext.DEFAULT);
NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsRequired);
for (int j = 0; j < original.length; j++) {
assertEquals("bpv=" + bpv, original[j], reader.get(j));
}
input.close();
}
}
private long[] randomLongs(MyRandom random, int bpv) {
int amount = random.nextInt(5000);
long longs[] = new long[amount];
for (int i = 0; i < longs.length; i++) {
longs[i] = random.nextLong(bpv);
}
return longs;
}
// java.util.Random only returns 48bits of randomness in nextLong...
static class MyRandom extends Random {
byte buffer[] = new byte[8];
ByteArrayDataInput input = new ByteArrayDataInput();
MyRandom(long seed) {
super(seed);
}
public synchronized long nextLong(int bpv) {
nextBytes(buffer);
input.reset(buffer);
long bits = input.readLong();
return bits >>> (64-bpv);
}
}
}

View File

@ -1123,7 +1123,7 @@ public class TestPackedInts extends LuceneTestCase {
final Directory dir = newDirectory();
final IndexOutput out = dir.createOutput("out.bin", IOContext.DEFAULT);
final BlockPackedWriter writer = new BlockPackedWriter(out, blockSize, PackedInts.COMPACT);
final BlockPackedWriter writer = new BlockPackedWriter(out, blockSize);
for (int i = 0; i < valueCount; ++i) {
assertEquals(i, writer.ord());
writer.add(values[i]);
@ -1247,7 +1247,7 @@ public class TestPackedInts extends LuceneTestCase {
final int blockSize = 1 << TestUtil.nextInt(random(), 20, 22);
final Directory dir = newDirectory();
final IndexOutput out = dir.createOutput("out.bin", IOContext.DEFAULT);
final BlockPackedWriter writer = new BlockPackedWriter(out, blockSize, PackedInts.COMPACT);
final BlockPackedWriter writer = new BlockPackedWriter(out, blockSize);
long value = random().nextInt() & 0xFFFFFFFFL;
long valueOffset = TestUtil.nextLong(random(), 0, valueCount - 1);
for (long i = 0; i < valueCount; ) {

View File

@ -32,7 +32,7 @@ import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -184,7 +184,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
* codec to use. */
protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer, IndexWriterConfig.OpenMode openMode) {
IndexWriterConfig iwc = new IndexWriterConfig(matchVersion, indexAnalyzer);
iwc.setCodec(new Lucene46Codec());
iwc.setCodec(new Lucene49Codec());
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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
/**
* Acts like {@link Lucene46Codec} but with additional asserts.
* Acts like {@link Lucene49Codec} 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 Lucene46Codec());
super("Asserting", new Lucene49Codec());
}
@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.lucene45.Lucene45DocValuesFormat;
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
import org.apache.lucene.index.AssertingAtomicReader;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
@ -39,10 +39,10 @@ import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LongBitSet;
/**
* Just like {@link Lucene45DocValuesFormat} but with additional asserts.
* Just like {@link Lucene49DocValuesFormat} but with additional asserts.
*/
public class AssertingDocValuesFormat extends DocValuesFormat {
private final DocValuesFormat in = new Lucene45DocValuesFormat();
private final DocValuesFormat in = new Lucene49DocValuesFormat();
public AssertingDocValuesFormat() {
super("Asserting");

View File

@ -28,7 +28,7 @@ import org.apache.lucene.codecs.diskdv.DiskNormsFormat;
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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
/** Codec that tries to use as little ram as possible because he spent all his money on beer */
// TODO: better name :)
@ -45,7 +45,7 @@ public class CheapBastardCodec extends FilterCodec {
private final NormsFormat norms = new DiskNormsFormat();
public CheapBastardCodec() {
super("CheapBastard", new Lucene46Codec());
super("CheapBastard", new Lucene49Codec());
}
@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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
/**
* A codec that uses {@link CompressingStoredFieldsFormat} for its stored
* fields and delegates to {@link Lucene46Codec} for everything else.
* fields and delegates to {@link Lucene49Codec} 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 Lucene46Codec());
super(name, new Lucene49Codec());
this.storedFieldsFormat = new CompressingStoredFieldsFormat(name, segmentSuffix, compressionMode, chunkSize);
this.termVectorsFormat = new CompressingTermVectorsFormat(name, segmentSuffix, compressionMode, chunkSize);
}

View File

@ -169,7 +169,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
data.writeLong(gcd);
data.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE, PackedInts.COMPACT);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
long value = nv == null ? 0 : nv.longValue();
writer.add((value - minValue) / gcd);
@ -181,7 +181,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
meta.writeVInt(PackedInts.VERSION_CURRENT);
data.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE, PackedInts.COMPACT);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
writer.add(nv == null ? 0 : nv.longValue());
}

View File

@ -19,10 +19,13 @@ package org.apache.lucene.codecs.lucene45;
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat;
import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosWriter;
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
import org.apache.lucene.util.LuceneTestCase;
/**
@ -46,4 +49,11 @@ public class Lucene45RWCodec extends Lucene45Codec {
public FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
private static final NormsFormat norms = new Lucene42NormsFormat();
@Override
public NormsFormat normsFormat() {
return norms;
}
}

View File

@ -0,0 +1,35 @@
package org.apache.lucene.codecs.lucene46;
/*
* 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.NormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
/**
* Read-write version of {@link Lucene46Codec} for testing.
*/
@SuppressWarnings("deprecation")
public class Lucene46RWCodec extends Lucene46Codec {
private static final NormsFormat norms = new Lucene42NormsFormat();
@Override
public NormsFormat normsFormat() {
return norms;
}
}

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.lucene46.Lucene46Codec}.
</body>
</html>

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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -1397,7 +1397,7 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
// TODO: would be better to use / delegate to the current
// Codec returned by getCodec()
iwc.setCodec(new Lucene46Codec() {
iwc.setCodec(new Lucene49Codec() {
@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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
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 Lucene46Codec();
otherCodec = new Lucene49Codec();
} else {
otherCodec = new SimpleTextCodec();
}

View File

@ -37,8 +37,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.lucene45.Lucene45DocValuesFormat;
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
import org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat;
@ -63,7 +63,7 @@ import org.apache.lucene.util.TestUtil;
* documents in different orders and the test will still be deterministic
* and reproducable.
*/
public class RandomCodec extends Lucene46Codec {
public class RandomCodec extends Lucene49Codec {
/** Shuffled list of postings formats to use for new mappings */
private List<PostingsFormat> formats = new ArrayList<>();
@ -150,7 +150,7 @@ public class RandomCodec extends Lucene46Codec {
new MemoryPostingsFormat(false, random.nextFloat()));
addDocValues(avoidCodecs,
new Lucene45DocValuesFormat(),
new Lucene49DocValuesFormat(),
new DiskDocValuesFormat(),
new MemoryDocValuesFormat(),
new SimpleTextDocValuesFormat(),

View File

@ -668,5 +668,200 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
fsdir.close();
}
// random access APIs
public void testRandomLong() throws Exception {
Directory dir = getDirectory(createTempDir("testLongs"));
IndexOutput output = dir.createOutput("longs", newIOContext(random()));
int num = TestUtil.nextInt(random(), 50, 3000);
long longs[] = new long[num];
for (int i = 0; i < longs.length; i++) {
longs[i] = TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE);
output.writeLong(longs[i]);
}
output.close();
// slice
IndexInput input = dir.openInput("longs", newIOContext(random()));
RandomAccessInput slice = input.randomAccessSlice(0, input.length());
for (int i = 0; i < longs.length; i++) {
assertEquals(longs[i], slice.readLong(i * 8));
}
// subslices
for (int i = 1; i < longs.length; i++) {
long offset = i * 8;
RandomAccessInput subslice = input.randomAccessSlice(offset, input.length() - offset);
for (int j = i; j < longs.length; j++) {
assertEquals(longs[j], subslice.readLong((j - i) * 8));
}
}
// with padding
for (int i = 0; i < 7; i++) {
String name = "longs-" + i;
IndexOutput o = dir.createOutput(name, newIOContext(random()));
byte junk[] = new byte[i];
random().nextBytes(junk);
o.writeBytes(junk, junk.length);
input.seek(0);
o.copyBytes(input, input.length());
o.close();
IndexInput padded = dir.openInput(name, newIOContext(random()));
RandomAccessInput whole = padded.randomAccessSlice(i, padded.length() - i);
for (int j = 0; j < longs.length; j++) {
assertEquals(longs[j], whole.readLong(j * 8));
}
padded.close();
}
input.close();
dir.close();
}
public void testRandomInt() throws Exception {
Directory dir = getDirectory(createTempDir("testInts"));
IndexOutput output = dir.createOutput("ints", newIOContext(random()));
int num = TestUtil.nextInt(random(), 50, 3000);
int ints[] = new int[num];
for (int i = 0; i < ints.length; i++) {
ints[i] = random().nextInt();
output.writeInt(ints[i]);
}
output.close();
// slice
IndexInput input = dir.openInput("ints", newIOContext(random()));
RandomAccessInput slice = input.randomAccessSlice(0, input.length());
for (int i = 0; i < ints.length; i++) {
assertEquals(ints[i], slice.readInt(i * 4));
}
// subslices
for (int i = 1; i < ints.length; i++) {
long offset = i * 4;
RandomAccessInput subslice = input.randomAccessSlice(offset, input.length() - offset);
for (int j = i; j < ints.length; j++) {
assertEquals(ints[j], subslice.readInt((j - i) * 4));
}
}
// with padding
for (int i = 0; i < 7; i++) {
String name = "ints-" + i;
IndexOutput o = dir.createOutput(name, newIOContext(random()));
byte junk[] = new byte[i];
random().nextBytes(junk);
o.writeBytes(junk, junk.length);
input.seek(0);
o.copyBytes(input, input.length());
o.close();
IndexInput padded = dir.openInput(name, newIOContext(random()));
RandomAccessInput whole = padded.randomAccessSlice(i, padded.length() - i);
for (int j = 0; j < ints.length; j++) {
assertEquals(ints[j], whole.readInt(j * 4));
}
padded.close();
}
input.close();
dir.close();
}
public void testRandomShort() throws Exception {
Directory dir = getDirectory(createTempDir("testShorts"));
IndexOutput output = dir.createOutput("shorts", newIOContext(random()));
int num = TestUtil.nextInt(random(), 50, 3000);
short shorts[] = new short[num];
for (int i = 0; i < shorts.length; i++) {
shorts[i] = (short) random().nextInt();
output.writeShort(shorts[i]);
}
output.close();
// slice
IndexInput input = dir.openInput("shorts", newIOContext(random()));
RandomAccessInput slice = input.randomAccessSlice(0, input.length());
for (int i = 0; i < shorts.length; i++) {
assertEquals(shorts[i], slice.readShort(i * 2));
}
// subslices
for (int i = 1; i < shorts.length; i++) {
long offset = i * 2;
RandomAccessInput subslice = input.randomAccessSlice(offset, input.length() - offset);
for (int j = i; j < shorts.length; j++) {
assertEquals(shorts[j], subslice.readShort((j - i) * 2));
}
}
// with padding
for (int i = 0; i < 7; i++) {
String name = "shorts-" + i;
IndexOutput o = dir.createOutput(name, newIOContext(random()));
byte junk[] = new byte[i];
random().nextBytes(junk);
o.writeBytes(junk, junk.length);
input.seek(0);
o.copyBytes(input, input.length());
o.close();
IndexInput padded = dir.openInput(name, newIOContext(random()));
RandomAccessInput whole = padded.randomAccessSlice(i, padded.length() - i);
for (int j = 0; j < shorts.length; j++) {
assertEquals(shorts[j], whole.readShort(j * 2));
}
padded.close();
}
input.close();
dir.close();
}
public void testRandomByte() throws Exception {
Directory dir = getDirectory(createTempDir("testBytes"));
IndexOutput output = dir.createOutput("bytes", newIOContext(random()));
int num = TestUtil.nextInt(random(), 50, 3000);
byte bytes[] = new byte[num];
random().nextBytes(bytes);
for (int i = 0; i < bytes.length; i++) {
output.writeByte(bytes[i]);
}
output.close();
// slice
IndexInput input = dir.openInput("bytes", newIOContext(random()));
RandomAccessInput slice = input.randomAccessSlice(0, input.length());
for (int i = 0; i < bytes.length; i++) {
assertEquals(bytes[i], slice.readByte(i));
}
// subslices
for (int i = 1; i < bytes.length; i++) {
long offset = i;
RandomAccessInput subslice = input.randomAccessSlice(offset, input.length() - offset);
for (int j = i; j < bytes.length; j++) {
assertEquals(bytes[j], subslice.readByte(j - i));
}
}
// with padding
for (int i = 0; i < 7; i++) {
String name = "bytes-" + i;
IndexOutput o = dir.createOutput(name, newIOContext(random()));
byte junk[] = new byte[i];
random().nextBytes(junk);
o.writeBytes(junk, junk.length);
input.seek(0);
o.copyBytes(input, input.length());
o.close();
IndexInput padded = dir.openInput(name, newIOContext(random()));
RandomAccessInput whole = padded.randomAccessSlice(i, padded.length() - i);
for (int j = 0; j < bytes.length; j++) {
assertEquals(bytes[j], whole.readByte(j));
}
padded.close();
}
input.close();
dir.close();
}
}

View File

@ -39,7 +39,8 @@ import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene46.Lucene46RWCodec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.RandomCodec;
@ -186,11 +187,19 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
} else if ("Lucene45".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
"random".equals(TEST_POSTINGSFORMAT) &&
"random".equals(TEST_DOCVALUESFORMAT) &&
randomVal == 5 &&
randomVal == 3 &&
!shouldAvoidCodec("Lucene45"))) {
codec = Codec.forName("Lucene45");
LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE = true;
assert codec instanceof Lucene45RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
} else if ("Lucene46".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
"random".equals(TEST_POSTINGSFORMAT) &&
"random".equals(TEST_DOCVALUESFORMAT) &&
randomVal == 4 &&
!shouldAvoidCodec("Lucene46"))) {
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 (("random".equals(TEST_POSTINGSFORMAT) == false) || ("random".equals(TEST_DOCVALUESFORMAT) == false)) {
// the user wired postings or DV: this is messy
// refactor into RandomCodec....
@ -206,12 +215,13 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
final DocValuesFormat dvFormat;
if ("random".equals(TEST_DOCVALUESFORMAT)) {
dvFormat = DocValuesFormat.forName("Lucene45");
// TODO: huh?
dvFormat = DocValuesFormat.forName("Lucene49");
} else {
dvFormat = DocValuesFormat.forName(TEST_DOCVALUESFORMAT);
}
codec = new Lucene46Codec() {
codec = new Lucene49Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;

View File

@ -46,7 +46,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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@ -682,7 +682,7 @@ public final class TestUtil {
if (LuceneTestCase.VERBOSE) {
System.out.println("forcing postings format to:" + format);
}
return new Lucene46Codec() {
return new Lucene49Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;
@ -700,7 +700,7 @@ public final class TestUtil {
if (LuceneTestCase.VERBOSE) {
System.out.println("forcing docvalues format to:" + format);
}
return new Lucene46Codec() {
return new Lucene49Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return format;

View File

@ -23,3 +23,4 @@ org.apache.lucene.codecs.lucene40.Lucene40RWCodec
org.apache.lucene.codecs.lucene41.Lucene41RWCodec
org.apache.lucene.codecs.lucene42.Lucene42RWCodec
org.apache.lucene.codecs.lucene45.Lucene45RWCodec
org.apache.lucene.codecs.lucene46.Lucene46RWCodec

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.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
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 Lucene46Codec() {
codec = new Lucene49Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);

View File

@ -22,7 +22,7 @@
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene41"/>
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Disk" />
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Lucene45" />
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Lucene49" />
<fieldType name="string" class="solr.StrField" />

View File

@ -55,10 +55,10 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
assertEquals("Disk", format.getDocValuesFormatForField(schemaField.getName()).getName());
schemaField = fields.get("string_memory_f");
assertEquals("Lucene45",
assertEquals("Lucene49",
format.getDocValuesFormatForField(schemaField.getName()).getName());
schemaField = fields.get("string_f");
assertEquals("Lucene45",
assertEquals("Lucene49",
format.getDocValuesFormatForField(schemaField.getName()).getName());
}
@ -80,7 +80,7 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
assertEquals("Disk", format.getDocValuesFormatForField("foo_disk").getName());
assertEquals("Disk", format.getDocValuesFormatForField("bar_disk").getName());
assertEquals("Lucene45", format.getDocValuesFormatForField("foo_memory").getName());
assertEquals("Lucene45", format.getDocValuesFormatForField("bar_memory").getName());
assertEquals("Lucene49", format.getDocValuesFormatForField("foo_memory").getName());
assertEquals("Lucene49", format.getDocValuesFormatForField("bar_memory").getName());
}
}