You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2022/05/12 08:36:28 UTC

[lucene] branch branch_9x updated: LUCENE-10536: Slightly better compression of doc values' terms dictionaries. (#838)

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 1677be091ec LUCENE-10536: Slightly better compression of doc values' terms dictionaries. (#838)
1677be091ec is described below

commit 1677be091ec9f3775d74bd927b8bfd4fea4d383d
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu May 12 10:32:58 2022 +0200

    LUCENE-10536: Slightly better compression of doc values' terms dictionaries. (#838)
    
    Doc values terms dictionaries keep the first term of each block uncompressed so
    that they can somewhat efficiently perform binary searches across blocks.
    Suffixes of the other 63 terms are compressed together using LZ4 to leverage
    redundancy across suffixes. This change improves compression a bit by using the
    first (uncompressed) term of each block as a dictionary when compressing
    suffixes of the 63 other terms. This helps with compressing the first few
    suffixes when there's not much context yet that can be leveraged to find
    duplicates.
---
 lucene/CHANGES.txt                                 |  4 +++
 .../codecs/lucene90/Lucene90DocValuesConsumer.java | 34 +++++++++++++---------
 .../codecs/lucene90/Lucene90DocValuesProducer.java | 14 +++++----
 .../lucene90/TestLucene90DocValuesFormat.java      | 30 +++++++++++++++++++
 4 files changed, 63 insertions(+), 19 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 550eaa742a4..a2dc12b169b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -82,6 +82,10 @@ Optimizations
 * LUCENE-8836: Speed up calls to TermsEnum#lookupOrd on doc values terms enums
   and sequences of increasing ords. (Bruno Roustant, Adrien Grand)
 
+* LUCENE-10536: Doc values terms dictionaries now use the first (uncompressed)
+  term of each block as a dictionary when compressing suffixes of the other 63
+  terms of the block. (Adrien Grand)
+
 * LUCENE-10542: FieldSource exists implementations can avoid value retrieval (Kevin Risden)
 
 * LUCENE-10534: MinFloatFunction / MaxFloatFunction exists check can be slow (Kevin Risden)
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
index 352a5104070..819c34e097e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
@@ -565,18 +565,26 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
 
     LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();
     ByteArrayDataOutput bufferedOutput = new ByteArrayDataOutput(termsDictBuffer);
+    int dictLength = 0;
 
     for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
       if ((ord & blockMask) == 0) {
-        if (bufferedOutput.getPosition() > 0) {
-          maxBlockLength =
-              Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
+        if (ord != 0) {
+          // flush the previous block
+          final int uncompressedLength =
+              compressAndGetTermsDictBlockLength(bufferedOutput, dictLength, ht);
+          maxBlockLength = Math.max(maxBlockLength, uncompressedLength);
           bufferedOutput.reset(termsDictBuffer);
         }
 
         writer.add(data.getFilePointer() - start);
+        // Write the first term both to the index output, and to the buffer where we'll use it as a
+        // dictionary for compression
         data.writeVInt(term.length);
         data.writeBytes(term.bytes, term.offset, term.length);
+        bufferedOutput = maybeGrowBuffer(bufferedOutput, term.length);
+        bufferedOutput.writeBytes(term.bytes, term.offset, term.length);
+        dictLength = term.length;
       } else {
         final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
         final int suffixLength = term.length - prefixLength;
@@ -598,9 +606,10 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
       ++ord;
     }
     // Compress and write out the last block
-    if (bufferedOutput.getPosition() > 0) {
-      maxBlockLength =
-          Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
+    if (bufferedOutput.getPosition() > dictLength) {
+      final int uncompressedLength =
+          compressAndGetTermsDictBlockLength(bufferedOutput, dictLength, ht);
+      maxBlockLength = Math.max(maxBlockLength, uncompressedLength);
     }
 
     writer.finish();
@@ -619,15 +628,12 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
   }
 
   private int compressAndGetTermsDictBlockLength(
-      ByteArrayDataOutput bufferedOutput, LZ4.FastCompressionHashTable ht) throws IOException {
-    int uncompressedLength = bufferedOutput.getPosition();
+      ByteArrayDataOutput bufferedOutput, int dictLength, LZ4.FastCompressionHashTable ht)
+      throws IOException {
+    int uncompressedLength = bufferedOutput.getPosition() - dictLength;
     data.writeVInt(uncompressedLength);
-    long before = data.getFilePointer();
-    LZ4.compress(termsDictBuffer, 0, uncompressedLength, data, ht);
-    int compressedLength = (int) (data.getFilePointer() - before);
-    // Block length will be used for creating buffer for decompression, one corner case is that
-    // compressed length might be bigger than un-compressed length, so just return the bigger one.
-    return Math.max(uncompressedLength, compressedLength);
+    LZ4.compressWithDictionary(termsDictBuffer, 0, dictLength, uncompressedLength, data, ht);
+    return uncompressedLength;
   }
 
   private ByteArrayDataOutput maybeGrowBuffer(ByteArrayDataOutput bufferedOutput, int termLength) {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java
index 5989334de38..cb5a27aa033 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java
@@ -1076,8 +1076,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
       indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength);
       term = new BytesRef(entry.maxTermLength);
 
+      // add the max term length for the dictionary
       // add 7 padding bytes can help decompression run faster.
-      int bufferSize = entry.maxBlockLength + LZ4_DECOMPRESSOR_PADDING;
+      int bufferSize = entry.maxBlockLength + entry.maxTermLength + LZ4_DECOMPRESSOR_PADDING;
       blockBuffer = new BytesRef(new byte[bufferSize], 0, bufferSize);
     }
 
@@ -1235,9 +1236,11 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
       if (offset < entry.termsDataLength - 1) {
         // Avoid decompress again if we are reading a same block.
         if (currentCompressedBlockStart != offset) {
-          int decompressLength = bytes.readVInt();
-          // Decompress the remaining of current block
-          LZ4.decompress(bytes, decompressLength, blockBuffer.bytes, 0);
+          blockBuffer.offset = term.length;
+          blockBuffer.length = bytes.readVInt();
+          // Decompress the remaining of current block, using the first term as a dictionary
+          System.arraycopy(term.bytes, 0, blockBuffer.bytes, 0, blockBuffer.offset);
+          LZ4.decompress(bytes, blockBuffer.length, blockBuffer.bytes, blockBuffer.offset);
           currentCompressedBlockStart = offset;
           currentCompressedBlockEnd = bytes.getFilePointer();
         } else {
@@ -1246,7 +1249,8 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
         }
 
         // Reset the buffer.
-        blockInput = new ByteArrayDataInput(blockBuffer.bytes, 0, blockBuffer.length);
+        blockInput =
+            new ByteArrayDataInput(blockBuffer.bytes, blockBuffer.offset, blockBuffer.length);
       }
     }
 
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
index 65b21f21165..2a11fefe0af 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
@@ -928,4 +928,34 @@ public class TestLucene90DocValuesFormat extends BaseCompressingDocValuesFormatT
       assertEquals(terms.get(i), te.term());
     }
   }
+
+  // Exercise the logic that leverages the first term of a block as a dictionary for suffixes of
+  // other terms
+  public void testTermsEnumDictionary() throws IOException {
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig();
+    RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    SortedDocValuesField field = new SortedDocValuesField("field", new BytesRef("abc0defghijkl"));
+    doc.add(field);
+    iwriter.addDocument(doc);
+    field.setBytesValue(new BytesRef("abc1defghijkl"));
+    iwriter.addDocument(doc);
+    field.setBytesValue(new BytesRef("abc2defghijkl"));
+    iwriter.addDocument(doc);
+    iwriter.forceMerge(1);
+    iwriter.close();
+
+    IndexReader reader = DirectoryReader.open(directory);
+    LeafReader leafReader = getOnlyLeafReader(reader);
+    SortedDocValues values = leafReader.getSortedDocValues("field");
+    TermsEnum termsEnum = values.termsEnum();
+    assertEquals(new BytesRef("abc0defghijkl"), termsEnum.next());
+    assertEquals(new BytesRef("abc1defghijkl"), termsEnum.next());
+    assertEquals(new BytesRef("abc2defghijkl"), termsEnum.next());
+    assertNull(termsEnum.next());
+
+    reader.close();
+    directory.close();
+  }
 }