You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/01/28 13:28:27 UTC

[GitHub] [lucene-solr] msokolov commented on a change in pull request #2213: LUCENE-9663: Adding compression to terms dict from SortedSet/Sorted DocValues

msokolov commented on a change in pull request #2213:
URL: https://github.com/apache/lucene-solr/pull/2213#discussion_r566092357



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
##########
@@ -791,6 +806,107 @@ private void addTermsDict(SortedSetDocValues values) throws IOException {
     writeTermsIndex(values);
   }
 
+  private void addCompressedTermsDict(SortedSetDocValues values) throws IOException {
+    final long size = values.getValueCount();
+    meta.writeVLong(size);
+    meta.writeInt(Lucene80DocValuesFormat.TERMS_DICT_BLOCK_LZ4_CODE);
+
+    ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
+    ByteBuffersIndexOutput addressOutput =
+        new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
+    meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+    long numBlocks =
+        (size + Lucene80DocValuesFormat.TERMS_DICT_BLOCK_LZ4_MASK)
+            >>> Lucene80DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
+    DirectMonotonicWriter writer =
+        DirectMonotonicWriter.getInstance(
+            meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();
+    ByteArrayDataOutput bufferedOutput = new ByteArrayDataOutput(termsDictBuffer);
+    long ord = 0;
+    long start = data.getFilePointer();
+    int maxLength = 0;
+    TermsEnum iterator = values.termsEnum();
+    int maxBlockLength = 0;
+    BytesRefBuilder previous = new BytesRefBuilder();
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      int termLength = term.length;
+      if ((ord & Lucene80DocValuesFormat.TERMS_DICT_BLOCK_LZ4_MASK) == 0) {
+        if (bufferedOutput.getPosition() > 0) {
+          int uncompressedLength = bufferedOutput.getPosition();
+          data.writeVInt(uncompressedLength);
+          maxBlockLength = Math.max(maxBlockLength, uncompressedLength);
+          long before = data.getFilePointer();
+          // Compress block
+          LZ4.compress(termsDictBuffer, 0, uncompressedLength, data, ht);
+          int compressedLength = (int) (data.getFilePointer() - before);
+          // Corner case: Compressed length might be bigger than un-compressed length.
+          maxBlockLength = Math.max(maxBlockLength, compressedLength);
+          bufferedOutput.reset(termsDictBuffer);
+        }
+
+        writer.add(data.getFilePointer() - start);
+        data.writeVInt(termLength);
+        data.writeBytes(term.bytes, term.offset, termLength);
+      } else {
+        final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
+        final int suffixLength = term.length - prefixLength;
+        assert suffixLength > 0; // terms are unique
+        int reservedSize = suffixLength + 11; // 1 byte + 2 vint.
+        bufferedOutput = maybeGrowBuffer(bufferedOutput, reservedSize);
+        bufferedOutput.writeByte(
+            (byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
+
+        if (prefixLength >= 15) {
+          bufferedOutput.writeVInt(prefixLength - 15);
+        }
+        if (suffixLength >= 16) {
+          bufferedOutput.writeVInt(suffixLength - 16);
+        }
+        bufferedOutput.writeBytes(term.bytes, term.offset + prefixLength, suffixLength);
+      }
+      maxLength = Math.max(maxLength, termLength);
+      previous.copyBytes(term);
+      ++ord;
+    }
+
+    // Compress and write out the last block
+    if (bufferedOutput.getPosition() > 0) {
+      int uncompressedLength = bufferedOutput.getPosition();
+      data.writeVInt(uncompressedLength);
+      maxBlockLength = Math.max(maxBlockLength, uncompressedLength);
+      long before = data.getFilePointer();
+      LZ4.compress(termsDictBuffer, 0, uncompressedLength, data, ht);
+      int compressedLength = (int) (data.getFilePointer() - before);
+      maxBlockLength = Math.max(maxBlockLength, compressedLength);
+    }
+
+    writer.finish();
+    meta.writeInt(maxLength);
+    // Write one more int for storing max block length. For compressed terms dict only.
+    meta.writeInt(maxBlockLength);
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+    start = data.getFilePointer();
+    addressBuffer.copyTo(data);
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+
+    // Now write the reverse terms index
+    writeTermsIndex(values);
+  }
+
+  private ByteArrayDataOutput maybeGrowBuffer(ByteArrayDataOutput bufferedOutput, int termLength) {
+    int pos = bufferedOutput.getPosition(), originalLength = termsDictBuffer.length;
+    if (pos + termLength >= originalLength - 1) {
+      int newLength = (originalLength + termLength) << 1;

Review comment:
       I don't completely understand the discussion here - line 851 is the call to ArrayUtil.grow? It looks like there is still a line 851 :) So -- I'll just say +1 to using Lucene's existing array-growing utility. It has an exponential growth that is < 2x, and unless this use case has some very special requirement, we should stick with that. If you feel the need to try something different, please substantiate it with tests on real data that show the value.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org