Return-Path: X-Original-To: apmail-lucene-commits-archive@www.apache.org Delivered-To: apmail-lucene-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id D0AB0D7B4 for ; Tue, 5 Feb 2013 20:53:25 +0000 (UTC) Received: (qmail 90742 invoked by uid 500); 5 Feb 2013 20:53:25 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 90734 invoked by uid 99); 5 Feb 2013 20:53:25 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 05 Feb 2013 20:53:25 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 05 Feb 2013 20:53:21 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id CE9C623888CD; Tue, 5 Feb 2013 20:53:00 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1442738 - in /lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene: codecs/lucene42/ search/similarities/ Date: Tue, 05 Feb 2013 20:53:00 -0000 To: commits@lucene.apache.org From: rmuir@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130205205300.CE9C623888CD@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: rmuir Date: Tue Feb 5 20:53:00 2013 New Revision: 1442738 URL: http://svn.apache.org/viewvc?rev=1442738&view=rev Log: tune default perf (fasterButMoreRam) Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?rev=1442738&r1=1442737&r2=1442738&view=diff ============================================================================== --- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original) +++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Tue Feb 5 20:53:00 2013 @@ -38,6 +38,7 @@ import org.apache.lucene.util.fst.Util; import org.apache.lucene.util.packed.BlockPackedWriter; import org.apache.lucene.util.packed.MonotonicBlockPackedWriter; import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.packed.PackedInts.FormatAndBits; /** * Writer for {@link Lucene42DocValuesFormat} @@ -51,14 +52,20 @@ class Lucene42DocValuesConsumer extends static final byte FST = 2; static final int BLOCK_SIZE = 4096; + + static final byte DELTA_COMPRESSED = 0; + static final byte TABLE_COMPRESSED = 1; + static final byte UNCOMPRESSED = 2; final IndexOutput data, meta; final int maxDoc; + final float acceptableOverheadRatio; - Lucene42DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + Lucene42DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, float acceptableOverheadRatio) throws IOException { + this.acceptableOverheadRatio = acceptableOverheadRatio; + maxDoc = state.segmentInfo.getDocCount(); boolean success = false; try { - maxDoc = state.segmentInfo.getDocCount(); String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); data = state.directory.createOutput(dataName, state.context); CodecUtil.writeHeader(data, dataCodec, VERSION_CURRENT); @@ -95,29 +102,37 @@ class Lucene42DocValuesConsumer extends } } - final long delta = maxValue - minValue; - if (uniqueValues != null && (delta < 0 || PackedInts.bitsRequired(uniqueValues.size()-1) < PackedInts.bitsRequired(delta))) { - // smaller to tableize + if (uniqueValues != null) { + // small number of unique values final int bitsPerValue = PackedInts.bitsRequired(uniqueValues.size()-1); - meta.writeByte((byte)1); // table-compressed - Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]); - final HashMap encode = new HashMap(); - data.writeVInt(decode.length); - for (int i = 0; i < decode.length; i++) { - data.writeLong(decode[i]); - encode.put(decode[i], i); - } - - meta.writeVInt(PackedInts.VERSION_CURRENT); - data.writeVInt(bitsPerValue); + FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(maxDoc, bitsPerValue, acceptableOverheadRatio); + if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) { + meta.writeByte(UNCOMPRESSED); // uncompressed + for (Number nv : values) { + data.writeByte((byte) nv.longValue()); + } + } else { + meta.writeByte(TABLE_COMPRESSED); // table-compressed + Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]); + final HashMap encode = new HashMap(); + data.writeVInt(decode.length); + for (int i = 0; i < decode.length; i++) { + data.writeLong(decode[i]); + encode.put(decode[i], i); + } - final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, PackedInts.Format.PACKED, maxDoc, bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE); - for(Number nv : values) { - writer.add(encode.get(nv)); + meta.writeVInt(PackedInts.VERSION_CURRENT); + data.writeVInt(formatAndBits.format.getId()); + data.writeVInt(formatAndBits.bitsPerValue); + + final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE); + for(Number nv : values) { + writer.add(encode.get(nv.longValue())); + } + writer.finish(); } - writer.finish(); } else { - meta.writeByte((byte)0); // delta-compressed + meta.writeByte(DELTA_COMPRESSED); // delta-compressed meta.writeVInt(PackedInts.VERSION_CURRENT); data.writeVInt(BLOCK_SIZE); Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java?rev=1442738&r1=1442737&r2=1442738&view=diff ============================================================================== --- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java (original) +++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java Tue Feb 5 20:53:00 2013 @@ -41,6 +41,9 @@ import org.apache.lucene.util.packed.Blo * the minimum value is encoded, and each entry is a delta from that minimum value. *
  • Table-compressed Numerics: when the number of unique values is very small, a lookup table * is written instead. Each per-document entry is instead the ordinal to this table. + *
  • Uncompressed Numerics: when all values would fit into a single byte, and the + * acceptableOverheadRatio would pack values into 8 bits per value anyway, they + * are written as absolute values (with no indirection or packing) for performance. *
  • Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length. * Each document's value can be addressed by maxDoc*length. *
  • Variable-width Binary: one large concatenated byte[] is written, along with end addresses @@ -83,6 +86,9 @@ import org.apache.lucene.util.packed.Blo * from the minimum value within the block. *
  • 1 --> 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. + *
  • 2 --> uncompressed. When the acceptableOverheadRatio parameter would upgrade the number + * of bits required to 8, and all values fit in a byte, these are written as absolute binary values + * for performance. * *

    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). @@ -93,11 +99,12 @@ import org.apache.lucene.util.packed.Blo *

    For DocValues field, this stores the actual per-document data (the heavy-lifting)

    *

    DocValues data (.dvd) --> Header,<NumericData | BinaryData | SortedData>NumFields

    *
      - *
    • NumericData --> DeltaCompressedNumerics | TableCompressedNumerics
    • + *
    • NumericData --> DeltaCompressedNumerics | TableCompressedNumerics | UncompressedNumerics
    • *
    • BinaryData --> {@link DataOutput#writeByte Byte}DataLength,Addresses
    • *
    • SortedData --> {@link FST FST<Int64>}
    • *
    • DeltaCompressedNumerics --> {@link BlockPackedWriter BlockPackedInts(blockSize=4096)}
    • *
    • TableCompressedNumerics --> TableSize,{@link DataOutput#writeLong Int64}TableSize,{@link PackedInts PackedInts}
    • + *
    • UncompressedNumerics --> {@link DataOutput#writeByte Byte}maxdoc
    • *
    • Addresses --> {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=4096)}
    • *
    * @@ -111,7 +118,8 @@ public final class Lucene42DocValuesForm @Override public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + // note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste) + return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, PackedInts.DEFAULT); } @Override Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1442738&r1=1442737&r2=1442738&view=diff ============================================================================== --- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original) +++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Tue Feb 5 20:53:00 2013 @@ -104,8 +104,10 @@ class Lucene42DocValuesProducer extends if (fieldType == Lucene42DocValuesConsumer.NUMBER) { NumericEntry entry = new NumericEntry(); entry.offset = meta.readLong(); - entry.tableized = meta.readByte() != 0; - entry.packedIntsVersion = meta.readVInt(); + entry.format = meta.readByte(); + if (entry.format != Lucene42DocValuesConsumer.UNCOMPRESSED) { + entry.packedIntsVersion = meta.readVInt(); + } numerics.put(fieldNumber, entry); } else if (fieldType == Lucene42DocValuesConsumer.BYTES) { BinaryEntry entry = new BinaryEntry(); @@ -143,21 +145,22 @@ class Lucene42DocValuesProducer extends private NumericDocValues loadNumeric(FieldInfo field) throws IOException { NumericEntry entry = numerics.get(field.number); data.seek(entry.offset); - if (entry.tableized) { + if (entry.format == Lucene42DocValuesConsumer.TABLE_COMPRESSED) { int size = data.readVInt(); final long decode[] = new long[size]; for (int i = 0; i < decode.length; i++) { decode[i] = data.readLong(); } + final int formatID = data.readVInt(); final int bitsPerValue = data.readVInt(); - final PackedInts.Reader reader = PackedInts.getReaderNoHeader(data, PackedInts.Format.PACKED, entry.packedIntsVersion, maxDoc, bitsPerValue); + final PackedInts.Reader reader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue); return new NumericDocValues() { @Override public long get(int docID) { return decode[(int)reader.get(docID)]; } }; - } else { + } else if (entry.format == Lucene42DocValuesConsumer.DELTA_COMPRESSED) { final int blockSize = data.readVInt(); final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false); return new NumericDocValues() { @@ -166,6 +169,17 @@ class Lucene42DocValuesProducer extends return reader.get(docID); } }; + } else if (entry.format == Lucene42DocValuesConsumer.UNCOMPRESSED) { + final byte bytes[] = new byte[maxDoc]; + data.readBytes(bytes, 0, bytes.length); + return new NumericDocValues() { + @Override + public long get(int docID) { + return bytes[docID]; + } + }; + } else { + throw new IllegalStateException(); } } @@ -279,7 +293,7 @@ class Lucene42DocValuesProducer extends static class NumericEntry { long offset; - boolean tableized; + byte format; int packedIntsVersion; } Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java?rev=1442738&r1=1442737&r2=1442738&view=diff ============================================================================== --- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java (original) +++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java Tue Feb 5 20:53:00 2013 @@ -24,12 +24,15 @@ import org.apache.lucene.codecs.DocValue import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.util.packed.PackedInts; /** * Lucene 4.2 score normalization format. *

    * NOTE: this uses the same format as {@link Lucene42DocValuesFormat} - * Numeric DocValues, but with different file extensions. + * Numeric DocValues, but with different file extensions, and passing + * {@link PackedInts#FASTEST} for uncompressed encoding: trading off + * space for performance. *

    * Files: *

      @@ -45,7 +48,8 @@ public final class Lucene42NormsFormat e @Override public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException { - return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + // note: we choose FASTEST here (otherwise our norms are half as big but 15% slower than previous lucene) + return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, PackedInts.FASTEST); } @Override Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java?rev=1442738&r1=1442737&r2=1442738&view=diff ============================================================================== --- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java (original) +++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Tue Feb 5 20:53:00 2013 @@ -772,23 +772,16 @@ public abstract class TFIDFSimilarity ex private final IDFStats stats; private final float weightValue; private final NumericDocValues norms; - private static final int SCORE_CACHE_SIZE = 32; - private float[] scoreCache = new float[SCORE_CACHE_SIZE]; ExactTFIDFDocScorer(IDFStats stats, NumericDocValues norms) throws IOException { this.stats = stats; this.weightValue = stats.value; this.norms = norms; - for (int i = 0; i < SCORE_CACHE_SIZE; i++) - scoreCache[i] = tf(i) * weightValue; } @Override public float score(int doc, int freq) { - final float raw = // compute tf(f)*weight - freq < SCORE_CACHE_SIZE // check cache - ? scoreCache[freq] // cache hit - : tf(freq)*weightValue; // cache miss + final float raw = tf(freq)*weightValue; // compute tf(f)*weight return norms == null ? raw : raw * decodeNormValue((byte)norms.get(doc)); // normalize for field }