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 2020/02/03 17:31:56 UTC

[GitHub] [lucene-solr] markharwood opened a new pull request #1234: Add compression for Binary doc value fields

markharwood opened a new pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234
 
 
   This PR stores groups of 32 doc values in LZ4 compressed blocks. 
   
   ### Write performance
   Test results for loading 680mb of log data (1.8m docs) are as follows:
   
   Branch | Load time (seconds, single thread) | Resulting index size (mb)
   ----|----|----
   master| 16| 680
   this PR| 11| 78
   
   ### Read performance
   Time taken to read 5,000 random doc IDs from above indices
   
   Branch | Read time (milliseconds, single thread) 
   ----|----
   master| 284
   this PR| 63
   
   On this particular test, the read + write speeds and resulting index size were all improved over the current master implementation. Obviously performance will vary with other tests with the main factors for changes being:
   * size of fields, 
   * compress-ability of field contents,
   * read access patterns (hitting same compressed blocks vs different ones)
   * variation from doc-to-doc in field value sizes,
   

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583529199
 
 
   In the case of content that can't be compressed, the compressed data will consist of the number of bytes, followed by the bytes. So decompressing consists of decoding the length and then reading the bytes. The only overhead compared to reading bytes directly is the decoding of the number of bytes, so I would believe that the overhead is rather small.
   
   I don't have a strong preference regarding whether this case should be handled explicitly or not. It's true that not having a special "not-compressed" case helps keep the logic simpler.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r374749291
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter  implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void  addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if(numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        data.writeVInt(numDocsInCurrentBlock);
+        for (int i = 0; i < numDocsInCurrentBlock; i++) {
 
 Review comment:
   Probably we could (later, separate issue) optimize writing these lengths -- often all docs will have the same length?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379294799
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
 
 Review comment:
   can we make `ht`, `tempBinaryOffsets`, `docLengths` final?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-581965010
 
 
   > my identity which looks to have been accidentally merged with a different "markh".
   
   LOL, that's crazy -- you should go introduce yourself to the other `markh` ;)

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] juanka588 commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
juanka588 commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r377579943
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,131 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
 
 Review comment:
   @jpountz we should use the same structure while writing the data, in that case you will see all the properties of the class instead of adding comments in the code

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r376532189
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,131 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
+    private int uncompressedBlockLength = 0;        
+    private int numDocsInBlock = 0;
+    private final byte[] uncompressedBlock;
+    private final BytesRef uncompressedBytesRef;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      uncompressedBytesRef = new BytesRef(uncompressedBlock);
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
+      int docInBlockId = docNumber % Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      assert docInBlockId < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      
+      
+      // already read and uncompressed?
+      if (blockId != lastBlockId) {
+        lastBlockId = blockId;
+        long blockStartOffset = addresses.get(blockId);
+        compressedData.seek(blockStartOffset);
+        
+        numDocsInBlock = compressedData.readVInt();
+        assert numDocsInBlock <= Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+        uncompressedDocEnds = new int[numDocsInBlock];
 
 Review comment:
   can we reuse the same array across blocks?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379307116
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      boolean success = false;
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+        success = true;
+      } finally {
+        if (success == false) {
+          IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        }
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        
+        // Optimisation - check if all lengths are same
+        boolean allLengthsSame = true && numDocsInCurrentBlock >0  ;
+        for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK && allLengthsSame; i++) {
+          if (i > 0 && docLengths[i] != docLengths[i-1]) {
 
 Review comment:
   if you're only doing it for `i>0`, let's make the loop start at `i=1`?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375827346
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,107 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
+    private int uncompressedBlockLength = 0;        
+    private int numDocsInBlock = 0;
+    private final byte[] uncompressedBlock;
+    private BytesRef uncompressedBytesRef;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      
 
 Review comment:
   we could initialize uncompressedBytesRef from the uncompressed block:
   `uncompressedBytesRef = new BytesRef(uncompressedBlock)`
   and avoid creating new BytesRefs over and over in `decode`

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379304369
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -182,10 +183,21 @@ private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
     entry.numDocsWithField = meta.readInt();
     entry.minLength = meta.readInt();
     entry.maxLength = meta.readInt();
-    if (entry.minLength < entry.maxLength) {
+    if ((version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED && entry.numDocsWithField >0)||  entry.minLength < entry.maxLength) {
 
 Review comment:
   ```suggestion
       if ((version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED && entry.numDocsWithField > 0) ||  entry.minLength < entry.maxLength) {
   ```

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375274563
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
 
 Review comment:
   we usually do this like that instead, which helps avoid catching Throwable
   
   ```
   boolean success = false;
   try {
     // write header
   } finally {
     if (success == false) {
       // close
     }
   }
   ```

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r376529195
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,131 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
+    private int uncompressedBlockLength = 0;        
+    private int numDocsInBlock = 0;
+    private final byte[] uncompressedBlock;
+    private final BytesRef uncompressedBytesRef;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      uncompressedBytesRef = new BytesRef(uncompressedBlock);
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
+      int docInBlockId = docNumber % Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      assert docInBlockId < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      
+      
+      // already read and uncompressed?
+      if (blockId != lastBlockId) {
+        lastBlockId = blockId;
+        long blockStartOffset = addresses.get(blockId);
+        compressedData.seek(blockStartOffset);
+        
+        numDocsInBlock = compressedData.readVInt();
+        assert numDocsInBlock <= Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+        uncompressedDocEnds = new int[numDocsInBlock];
+        uncompressedBlockLength = 0;        
+
+        int onlyLength = -1;
+        for (int i = 0; i < numDocsInBlock; i++) {
+          if (i == 0) {
+            // The first length value is special. It is shifted and has a bit to denote if
+            // all other values are the same length
+            int lengthPlusSameInd = compressedData.readVInt();
+            int sameIndicator = lengthPlusSameInd & 1;
+            int firstValLength = lengthPlusSameInd >>1;
 
 Review comment:
   Since you are stealing a bit, we should do an unsigned shift (`>>>`) instead.
   
   This would never be a problem in practice, but imagine than the length was a 31-bits integer. Shifting by one bit on the left at index time would make this number negative. So here we need an unsigned shift rather than a signed shift that preserves the sign.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-582866319
 
 
   >And how can indexing and searching get so much faster when compress/decompress is in the path!
   
   I tried benchmarking some straight-forward file read and write operations (no Lucene) and couldn't show LZ4 compression being faster (although it wasn't that much slower).
   
   Maybe the rate-limited merging in Lucene plays a part and size therefore matters in that context?
   

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r374750204
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter  implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void  addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if(numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        data.writeVInt(numDocsInCurrentBlock);
+        for (int i = 0; i < numDocsInCurrentBlock; i++) {
+          data.writeVInt(docLengths[i]);
+        }
+        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
+        LZ4.compress(block,  0, uncompressedBlockLength, data, ht);
+        numDocsInCurrentBlock = 0;
+        uncompressedBlockLength = 0;
+        maxPointer = data.getFilePointer();
+        tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
+      }
+    }
+    
+    void writeMetaData() throws IOException {
+      if (blockAddressesStart < 0 ) {
+        return;
+      }
+      
+      long startDMW = data.getFilePointer();
+      meta.writeLong(startDMW);
+      
+      meta.writeInt(totalChunks);
+      meta.writeInt(maxUncompressedBlockLength);
       meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+      
+    
+      CodecUtil.writeFooter(tempBinaryOffsets);
+      IOUtils.close(tempBinaryOffsets);             
+      //write the compressed block offsets info to the meta file by reading from temp file
+      try (ChecksumIndexInput filePointersIn = state.directory.openChecksumInput(tempBinaryOffsets.getName(), IOContext.READONCE)) {
+        CodecUtil.checkHeader(filePointersIn, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT,
+          Lucene80DocValuesFormat.VERSION_CURRENT);
+        Throwable priorE = null;
+        try {
+          final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance(meta, data, totalChunks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+          long fp = blockAddressesStart;
+          for (int i = 0; i < totalChunks; ++i) {
+            filePointers.add(fp);
+            fp += filePointersIn.readVLong();
+          }
+          if (maxPointer < fp) {
+            throw new CorruptIndexException("File pointers don't add up", filePointersIn);
 
 Review comment:
   Can you include `maxPointer` and `fp` in this exception message?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375252907
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
 
 Review comment:
   ```suggestion
       byte[] block = new byte [1024 * 16];
   ```

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375252836
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
 
 Review comment:
   we usually don't let spaces between the type of array elements and `[]`
   
   ```suggestion
       int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
   ```

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r377621003
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      boolean success = false;
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+        success = true;
+      } finally {
+        if (success == false) {
+          IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        }
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        
+        // Optimisation - check if all lengths are same
+        boolean allLengthsSame = true && numDocsInCurrentBlock >0  ;
+        for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK && allLengthsSame; i++) {
+          if (i > 0 && docLengths[i] != docLengths[i-1]) {
+            allLengthsSame = false;
+          }
+        }
+        if (allLengthsSame) {
+            // Only write one value shifted. Steal a bit to indicate all other lengths are the same
+            int onlyOneLength = (docLengths[0] <<1) | 1;
+            data.writeVInt(onlyOneLength);
+        } else {
+          for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) {
+            if(i == 0) {
+              // Write first value shifted and steal a bit to indicate other lengths are to follow
+              int multipleLengths = (docLengths[0] <<1);
+              data.writeVInt(multipleLengths);              
+            } else {
+              data.writeVInt(docLengths[i]);
+            }
+          }
+        }
+        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
+        LZ4.compress(block, 0, uncompressedBlockLength, data, ht);
+        numDocsInCurrentBlock = 0;
+        // Ensure initialized with zeroes because full array is always written
+        Arrays.fill(docLengths, 0);
+        uncompressedBlockLength = 0;
+        maxPointer = data.getFilePointer();
+        tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
+      }
+    }
+    
+    void writeMetaData() throws IOException {
+      if (blockAddressesStart < 0 ) {
+        return;
+      }
+      
+      long startDMW = data.getFilePointer();
+      meta.writeLong(startDMW);
+      
+      meta.writeVInt(totalChunks);
+      meta.writeVInt(Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK);
+      meta.writeVInt(maxUncompressedBlockLength);
       meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+      
+    
+      CodecUtil.writeFooter(tempBinaryOffsets);
+      IOUtils.close(tempBinaryOffsets);             
+      //write the compressed block offsets info to the meta file by reading from temp file
+      try (ChecksumIndexInput filePointersIn = state.directory.openChecksumInput(tempBinaryOffsets.getName(), IOContext.READONCE)) {
+        CodecUtil.checkHeader(filePointersIn, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT,
+          Lucene80DocValuesFormat.VERSION_CURRENT);
+        Throwable priorE = null;
+        try {
+          final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance(meta, data, totalChunks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+          long fp = blockAddressesStart;
+          for (int i = 0; i < totalChunks; ++i) {
+            filePointers.add(fp);
+            fp += filePointersIn.readVLong();
+          }
+          if (maxPointer < fp) {
+            throw new CorruptIndexException("File pointers don't add up ("+fp+" vs expected "+maxPointer+")", filePointersIn);
+          }
+          filePointers.finish();
+        } catch (Throwable e) {
+          priorE = e;
+        } finally {
+          CodecUtil.checkFooter(filePointersIn, priorE);
+        }
+      }
+      // Write the length of the DMW block in the data 
+      meta.writeLong(data.getFilePointer() - startDMW);
+    }
 
-      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
-      long addr = 0;
-      writer.add(addr);
-      values = valuesProducer.getBinary(field);
+    @Override
+    public void close() throws IOException {
+      if (tempBinaryOffsets != null) {
+        IOUtils.close(tempBinaryOffsets);             
+        state.directory.deleteFile(tempBinaryOffsets.getName());
+        tempBinaryOffsets = null;
+      }
+    }
+    
+  }
+  
+
+  @Override
+  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene80DocValuesFormat.BINARY);
+
+    try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()){
+      BinaryDocValues values = valuesProducer.getBinary(field);
+      long start = data.getFilePointer();
+      meta.writeLong(start); // dataOffset
 
 Review comment:
   I like the idea but would prefer doing it in a separate PR.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379305128
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +757,125 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private final int []uncompressedDocStarts;
+    private int uncompressedBlockLength = 0;        
+    private final byte[] uncompressedBlock;
+    private final BytesRef uncompressedBytesRef;
+    private final int docsPerChunk;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize, int docsPerChunk) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      uncompressedBytesRef = new BytesRef(uncompressedBlock);
+      this.docsPerChunk = docsPerChunk;
+      uncompressedDocStarts = new int[docsPerChunk + 1];
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
 
 Review comment:
   let's use the shift from the BinaryEntry instead of the constant?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379473860
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +757,125 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private final int []uncompressedDocStarts;
+    private int uncompressedBlockLength = 0;        
+    private final byte[] uncompressedBlock;
+    private final BytesRef uncompressedBytesRef;
+    private final int docsPerChunk;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize, int docsPerChunk) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      uncompressedBytesRef = new BytesRef(uncompressedBlock);
+      this.docsPerChunk = docsPerChunk;
+      uncompressedDocStarts = new int[docsPerChunk + 1];
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
 
 Review comment:
   I think so.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375275497
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
 
 Review comment:
   it looks like we could set `blockAddressesStart` in the constructor instead?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375277898
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        data.writeVInt(numDocsInCurrentBlock);
+        for (int i = 0; i < numDocsInCurrentBlock; i++) {
+          data.writeVInt(docLengths[i]);
+        }
+        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
+        LZ4.compress(block,  0, uncompressedBlockLength, data, ht);
 
 Review comment:
   ```suggestion
           LZ4.compress(block, 0, uncompressedBlockLength, data, ht);
   ```

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood edited a comment on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood edited a comment on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583539216
 
 
   >Strange that Mark would measure 4x slowdown from decoding the lengths... Perhaps the random bytes are not totally incompressible, just barely compressible?
   
   I may have been too hasty in that reply - I've not been able to reproduce that and the timings are very similar in the additional tests I've done so echo what @jpountz expects. My first (faster) run had random bytes selected in the range 0-20 and not the 0-127 range where I'm seeing parity

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375914836
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        data.writeVInt(numDocsInCurrentBlock);
+        for (int i = 0; i < numDocsInCurrentBlock; i++) {
+          data.writeVInt(docLengths[i]);
+        }
+        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
+        LZ4.compress(block,  0, uncompressedBlockLength, data, ht);
+        numDocsInCurrentBlock = 0;
+        uncompressedBlockLength = 0;
+        maxPointer = data.getFilePointer();
+        tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
+      }
+    }
+    
+    void writeMetaData() throws IOException {
+      if (blockAddressesStart < 0 ) {
 
 Review comment:
   Looks to be when merges clear out deleted docs leaving no values.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379295432
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
 
 Review comment:
   Depending on the data that will be indexed it's very hard to know what is the right initial size here. Maybe start with an empty array? This will also give increase confidence that the resizing logic works.
   
   ```suggestion
       byte[] block = BytesRef.EMPTY_BYTES;
   ```

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379306326
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -182,10 +183,21 @@ private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
     entry.numDocsWithField = meta.readInt();
     entry.minLength = meta.readInt();
     entry.maxLength = meta.readInt();
-    if (entry.minLength < entry.maxLength) {
+    if ((version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED && entry.numDocsWithField >0)||  entry.minLength < entry.maxLength) {
       entry.addressesOffset = meta.readLong();
+
+      // Old count of uncompressed addresses 
+      long numAddresses = entry.numDocsWithField + 1L;
+      // New count of compressed addresses - the number of compresseed blocks
+      if (version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED) {
+        entry.numCompressedChunks = meta.readVInt();
+        entry.docsPerChunk = meta.readVInt();
 
 Review comment:
   maybe this should be the "shift" instead of the number of docs per chunk, so that you you directly have both the shift (as-is) and the mask `((1 << shift) - 1)`

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583313015
 
 
   I've reclaimed my Jira log-in and opened https://issues.apache.org/jira/browse/LUCENE-9211

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] juanka588 commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
juanka588 commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r377545909
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      boolean success = false;
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+        success = true;
+      } finally {
+        if (success == false) {
+          IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        }
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        
+        // Optimisation - check if all lengths are same
+        boolean allLengthsSame = true && numDocsInCurrentBlock >0  ;
+        for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK && allLengthsSame; i++) {
+          if (i > 0 && docLengths[i] != docLengths[i-1]) {
+            allLengthsSame = false;
+          }
+        }
+        if (allLengthsSame) {
+            // Only write one value shifted. Steal a bit to indicate all other lengths are the same
+            int onlyOneLength = (docLengths[0] <<1) | 1;
+            data.writeVInt(onlyOneLength);
+        } else {
+          for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) {
+            if(i == 0) {
+              // Write first value shifted and steal a bit to indicate other lengths are to follow
+              int multipleLengths = (docLengths[0] <<1);
+              data.writeVInt(multipleLengths);              
+            } else {
+              data.writeVInt(docLengths[i]);
+            }
+          }
+        }
+        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
+        LZ4.compress(block, 0, uncompressedBlockLength, data, ht);
+        numDocsInCurrentBlock = 0;
+        // Ensure initialized with zeroes because full array is always written
+        Arrays.fill(docLengths, 0);
+        uncompressedBlockLength = 0;
+        maxPointer = data.getFilePointer();
+        tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
+      }
+    }
+    
+    void writeMetaData() throws IOException {
+      if (blockAddressesStart < 0 ) {
+        return;
+      }
+      
+      long startDMW = data.getFilePointer();
+      meta.writeLong(startDMW);
+      
+      meta.writeVInt(totalChunks);
+      meta.writeVInt(Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK);
+      meta.writeVInt(maxUncompressedBlockLength);
       meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+      
+    
+      CodecUtil.writeFooter(tempBinaryOffsets);
+      IOUtils.close(tempBinaryOffsets);             
+      //write the compressed block offsets info to the meta file by reading from temp file
+      try (ChecksumIndexInput filePointersIn = state.directory.openChecksumInput(tempBinaryOffsets.getName(), IOContext.READONCE)) {
+        CodecUtil.checkHeader(filePointersIn, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT,
+          Lucene80DocValuesFormat.VERSION_CURRENT);
+        Throwable priorE = null;
+        try {
+          final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance(meta, data, totalChunks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+          long fp = blockAddressesStart;
+          for (int i = 0; i < totalChunks; ++i) {
+            filePointers.add(fp);
+            fp += filePointersIn.readVLong();
+          }
+          if (maxPointer < fp) {
+            throw new CorruptIndexException("File pointers don't add up ("+fp+" vs expected "+maxPointer+")", filePointersIn);
+          }
+          filePointers.finish();
+        } catch (Throwable e) {
+          priorE = e;
+        } finally {
+          CodecUtil.checkFooter(filePointersIn, priorE);
+        }
+      }
+      // Write the length of the DMW block in the data 
+      meta.writeLong(data.getFilePointer() - startDMW);
+    }
 
-      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
-      long addr = 0;
-      writer.add(addr);
-      values = valuesProducer.getBinary(field);
+    @Override
+    public void close() throws IOException {
+      if (tempBinaryOffsets != null) {
+        IOUtils.close(tempBinaryOffsets);             
+        state.directory.deleteFile(tempBinaryOffsets.getName());
+        tempBinaryOffsets = null;
+      }
+    }
+    
+  }
+  
+
+  @Override
+  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene80DocValuesFormat.BINARY);
+
+    try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()){
+      BinaryDocValues values = valuesProducer.getBinary(field);
+      long start = data.getFilePointer();
+      meta.writeLong(start); // dataOffset
+      int numDocsWithField = 0;
+      int minLength = Integer.MAX_VALUE;
+      int maxLength = 0;
       for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-        addr += values.binaryValue().length;
-        writer.add(addr);
+        numDocsWithField++;
+        BytesRef v = values.binaryValue();      
+        blockWriter.addDoc(doc, v);      
+        int length = v.length;      
+        minLength = Math.min(length, minLength);
+        maxLength = Math.max(length, maxLength);
       }
-      writer.finish();
-      meta.writeLong(data.getFilePointer() - start);
+      blockWriter.flushData();
+
+      assert numDocsWithField <= maxDoc;
+      meta.writeLong(data.getFilePointer() - start); // dataLength
+
+      if (numDocsWithField == 0) {
+        meta.writeLong(-2); // docsWithFieldOffset
+        meta.writeLong(0L); // docsWithFieldLength
+        meta.writeShort((short) -1); // jumpTableEntryCount
+        meta.writeByte((byte) -1);   // denseRankPower
+      } else if (numDocsWithField == maxDoc) {
+        meta.writeLong(-1); // docsWithFieldOffset
+        meta.writeLong(0L); // docsWithFieldLength
+        meta.writeShort((short) -1); // jumpTableEntryCount
+        meta.writeByte((byte) -1);   // denseRankPower
+      } else {
+        long offset = data.getFilePointer();
+        meta.writeLong(offset); // docsWithFieldOffset
+        values = valuesProducer.getBinary(field);
+        final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+        meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
+        meta.writeShort(jumpTableEntryCount);
+        meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+      }
 
 Review comment:
   Currently I'm working in a refactor of this code by having a doc id set iterator serializer capable to provide the correct instance based on the stored metadata. As you might see this is quite repetitive for the other fields

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r374748519
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter  implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void  addDoc(int doc, BytesRef v) throws IOException {
 
 Review comment:
   Remove one of the spaces after `void` before `addDoc`?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-581966699
 
 
   @markharwood how can we reproduce these benchmarks?  What were the log data documents storing as `BINARY` doc values fields?
   
   And how can indexing and searching get so much faster when compress/decompress is in the path!  These numbers seem a little too good to believe ... but if they hold up, that's incredible.  Lucene's default codec doesn't do any compression for `BINARY` doc values so it'd be great to add that, especially with fun issues looking to give [vectors first class support](http://issues.apache.org/jira/browse/LUCENE-9004) [in Lucene](https://issues.apache.org/jira/browse/LUCENE-9136) (backed by `BINARY` doc values, I think).

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379297803
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
 
 Review comment:
   Have you found what this `something else` is?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375927967
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
 
 Review comment:
   It was about optimizing for the case that all values have the same length. In that case we could still one bit of the first length to mean that all values have the same length for instance?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375278323
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        data.writeVInt(numDocsInCurrentBlock);
+        for (int i = 0; i < numDocsInCurrentBlock; i++) {
+          data.writeVInt(docLengths[i]);
+        }
+        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
+        LZ4.compress(block,  0, uncompressedBlockLength, data, ht);
+        numDocsInCurrentBlock = 0;
+        uncompressedBlockLength = 0;
+        maxPointer = data.getFilePointer();
+        tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
+      }
+    }
+    
+    void writeMetaData() throws IOException {
+      if (blockAddressesStart < 0 ) {
 
 Review comment:
   this only happens if there are no values? when do we run into this condition?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375252133
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -61,11 +66,13 @@
 
   IndexOutput data, meta;
   final int maxDoc;
+  private SegmentWriteState state;
 
 Review comment:
   make it final?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375273736
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter  implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void  addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if(numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        data.writeVInt(numDocsInCurrentBlock);
+        for (int i = 0; i < numDocsInCurrentBlock; i++) {
 
 Review comment:
   +1

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375922373
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
 
 Review comment:
   What was the "+1" comment for line 407 about?
   I've seen encoding elsewhere that have n+1 offsets to record start of each value and the last offset is effectively the end of the last value. In this scenario I'm writing n value lengths. 

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379463675
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -182,10 +183,21 @@ private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
     entry.numDocsWithField = meta.readInt();
     entry.minLength = meta.readInt();
     entry.maxLength = meta.readInt();
-    if (entry.minLength < entry.maxLength) {
+    if ((version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED && entry.numDocsWithField >0)||  entry.minLength < entry.maxLength) {
       entry.addressesOffset = meta.readLong();
+
+      // Old count of uncompressed addresses 
+      long numAddresses = entry.numDocsWithField + 1L;
+      // New count of compressed addresses - the number of compresseed blocks
+      if (version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED) {
+        entry.numCompressedChunks = meta.readVInt();
+        entry.docsPerChunk = meta.readVInt();
 
 Review comment:
   Ah - ignore my previous comment.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-582367395
 
 
   Thanks for looking at this, Mike.
   
   >LOL, that's crazy -- you should go introduce yourself to the other markh ;)
   
   I already reached out and we're working out the divorce proceedings :)
   
   >@markharwood how can we reproduce these benchmarks? What were the log data documents storing as BINARY doc values fields?
   
   These were elasticsearch log file entries - so each value was a string which could be something short like  `[instance-0000000048] users file [/app/config/users] changed. updating users... )` or an error with a whole stack trace.
   My test rig is [here](https://gist.github.com/markharwood/724009754c89e7f245625120e71f60d7) if you want to try with some other data files
   
   >And how can indexing and searching get so much faster when compress/decompress is in the path!
   
   This was a test on my macbook with SSD and encrypted FS so perhaps not the best benchmarking setup. Maybe just writing more bytes = more overhead with the OS-level encryption?
   
   >I think our testing of BINARY doc values may not be great ... maybe add a randomized test that sometimes stores very compressible and very incompressible, large, BINARY doc values?
   
   Will do. @jimczi has suggested adding support for storing without compression when the content doesn't compress well. I guess that can be a combination of :
   1) A fast heuristic - e.g. if max value length for each of the docs in a block <=2 then store without compression and 
   2) "Try it and see" compression - buffer compression output to byte array and only write compressed form to disk if size is less than the uncompressed input
   
   

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] juanka588 commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
juanka588 commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r377566543
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesFormat.java
 ##########
 @@ -151,7 +151,8 @@ public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOExcepti
   static final String META_CODEC = "Lucene80DocValuesMetadata";
   static final String META_EXTENSION = "dvm";
   static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+  static final int VERSION_BIN_COMPRESSED = 1;  
 
 Review comment:
   This could be potentially in the BinaryDocValuesFormat class

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood merged pull request #1234: LUCENE-9211 Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood merged pull request #1234: LUCENE-9211 Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234
 
 
   

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-581843351
 
 
   FYI - related Jira issue is pending. I can't log in until Jira admins recover my identity which looks to have been accidentally merged with a different "markh". 

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r374745811
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter  implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void  addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if(numDocsInCurrentBlock > 0) {
 
 Review comment:
   Add space after `if` before `(`?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379298761
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      boolean success = false;
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+        success = true;
+      } finally {
+        if (success == false) {
+          IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        }
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        
+        // Optimisation - check if all lengths are same
+        boolean allLengthsSame = true && numDocsInCurrentBlock >0  ;
 
 Review comment:
   The second condition is necessary true given the parent if statement.
   ```suggestion
           boolean allLengthsSame = true;
   ```

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583449275
 
 
   There was a suggestion from @jimczi that we fall back to writing raw data if content doesn't compress well. I'm not sure this logic is worth developing for the reasons outlined below:
   
   I wrote a [compression buffer](https://gist.github.com/markharwood/91cc8d96d6611ad97df11f244b1b1d0f) to see what the compression algo outputs before deciding whether to write the compressed or  raw data to disk.
   I tested with the most uncompressible content I could imagine:
   
       public static void fillRandom(byte[] buffer, int length) {
           for (int i = 0; i < length; i++) {
               buffer[i] =  (byte) (Math.random() * Byte.MAX_VALUE);
           }
       } 
   
   The LZ4 compressed versions of this content were only marginally bigger than their raw counterparts (adding 0.4% overhead to the original content e.g. 96,921 compressed vs 96,541 raw bytes).
   On that basis I'm not sure if it's worth doubling the memory costs of the indexing logic (we would require a temporary output buffer that is at least the same size as the raw data being compressed) and additional byte shuffling.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583529462
 
 
   >Did you also test read performance in this incompressible case?
   
   Just tried it and it does look 4x faster reading raw random bytes Vs compressed random bytes
   

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583536606
 
 
   @msokolov FWIW LZ4 only removes duplicate strings from a stream: when it finds one it inserts a reference to a previous sequence of bytes. In the special case that the content in incompressible, the LZ4 compressed data just consists of the number of bytes followed by the bytes, so the only overhead compared to reading the bytes directly is the decoding of the number of bytes, which should be rather low.
   
   I don't have a preference regarding whether we should have an explicit "not-compressed" case, but I understand how not having one helps keep things simpler.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r376528169
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,131 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
+    private int uncompressedBlockLength = 0;        
+    private int numDocsInBlock = 0;
+    private final byte[] uncompressedBlock;
+    private final BytesRef uncompressedBytesRef;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      uncompressedBytesRef = new BytesRef(uncompressedBlock);
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
+      int docInBlockId = docNumber % Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      assert docInBlockId < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      
+      
+      // already read and uncompressed?
+      if (blockId != lastBlockId) {
+        lastBlockId = blockId;
+        long blockStartOffset = addresses.get(blockId);
+        compressedData.seek(blockStartOffset);
+        
+        numDocsInBlock = compressedData.readVInt();
 
 Review comment:
   do we really need to record the number of documents in the block? It should be 32 for all blocks except for the last one? Maybe at index-time we could append dummy values to the last block to make sure it has 32 values too, and we wouldn't need this vInt anymore?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375903347
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
 
 Review comment:
   I tried that and it didn't work - something else was writing to data in between constructor and addDoc calls

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379298212
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      boolean success = false;
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+        success = true;
+      } finally {
+        if (success == false) {
+          IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        }
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        
+        // Optimisation - check if all lengths are same
+        boolean allLengthsSame = true && numDocsInCurrentBlock >0  ;
+        for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK && allLengthsSame; i++) {
+          if (i > 0 && docLengths[i] != docLengths[i-1]) {
+            allLengthsSame = false;
+          }
+        }
+        if (allLengthsSame) {
+            // Only write one value shifted. Steal a bit to indicate all other lengths are the same
+            int onlyOneLength = (docLengths[0] <<1) | 1;
+            data.writeVInt(onlyOneLength);
+        } else {
+          for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) {
+            if(i == 0) {
 
 Review comment:
   ```suggestion
               if (i == 0) {
   ```

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379463440
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +757,125 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private final int []uncompressedDocStarts;
+    private int uncompressedBlockLength = 0;        
+    private final byte[] uncompressedBlock;
+    private final BytesRef uncompressedBytesRef;
+    private final int docsPerChunk;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize, int docsPerChunk) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      uncompressedBytesRef = new BytesRef(uncompressedBlock);
+      this.docsPerChunk = docsPerChunk;
+      uncompressedDocStarts = new int[docsPerChunk + 1];
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
 
 Review comment:
   I guess that means I should serialize the shift value rather the absolute number of docs per block?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r374747203
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,107 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
+    private int uncompressedBlockLength = 0;        
+    private int numDocsInBlock = 0;
+    private final byte[] uncompressedBlock;
+    private BytesRef uncompressedBytesRef;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
+      int docInBlockId = docNumber % Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      assert docInBlockId < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      
+      
+      // already read and uncompressed?
+      if ( blockId != lastBlockId) {
+        lastBlockId = blockId;
+        long blockStartOffset = addresses.get(blockId);
+        compressedData.seek(blockStartOffset);
+        
+        numDocsInBlock = compressedData.readVInt();
+        assert numDocsInBlock <= Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+        uncompressedDocEnds = new int[numDocsInBlock];
+        uncompressedBlockLength = 0;        
+        for (int i = 0; i < numDocsInBlock; i++) {
+          uncompressedBlockLength += compressedData.readVInt();
+          uncompressedDocEnds[i] = uncompressedBlockLength;
+        }
+        
+        if (uncompressedBlockLength == 0) {
+          uncompressedBytesRef = new BytesRef(BytesRef.EMPTY_BYTES);
+        } else {
+          assert uncompressedBlockLength <= uncompressedBlock.length;
+          LZ4.decompress(compressedData, uncompressedBlockLength, uncompressedBlock, 0);
+          uncompressedBytesRef = new BytesRef(uncompressedBlock);
+        }
+      }
+      
+      // Position the Bytes ref to the relevant part of the uncompressed block
 
 Review comment:
   s/`Bytes ref`/`BytesRef`?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583539216
 
 
   >Strange that Mark would measure 4x slowdown from decoding the lengths... Perhaps the random bytes are not totally incompressible, just barely compressible?
   
   I may have been too hasty in that reply - I've not been able to reproduce that and the timings are very similar in the additional tests I've done so echo what @jpountz expects

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r375974370
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter  implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void  addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if(numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        data.writeVInt(numDocsInCurrentBlock);
+        for (int i = 0; i < numDocsInCurrentBlock; i++) {
 
 Review comment:
   Done

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] markharwood edited a comment on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
markharwood edited a comment on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583539216
 
 
   >Strange that Mark would measure 4x slowdown from decoding the lengths... Perhaps the random bytes are not totally incompressible, just barely compressible?
   
   I may have been too hasty in that reply - I've not been able to reproduce that and the raw vs compressed timings are very similar in the additional tests I've done so echo what @jpountz expects. My first (faster) run had random bytes selected in the range 0-20 and not the 0-127 range where I'm seeing parity

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
msokolov commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583538389
 
 
   Strange that Mark would measure 4x slowdown from decoding the lengths... Perhaps the random bytes are not totally incompressible, just barely compressible? 

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r376527753
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,131 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
+    private int uncompressedBlockLength = 0;        
+    private int numDocsInBlock = 0;
+    private final byte[] uncompressedBlock;
+    private final BytesRef uncompressedBytesRef;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      uncompressedBytesRef = new BytesRef(uncompressedBlock);
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
+      int docInBlockId = docNumber % Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      assert docInBlockId < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      
+      
+      // already read and uncompressed?
+      if (blockId != lastBlockId) {
+        lastBlockId = blockId;
+        long blockStartOffset = addresses.get(blockId);
+        compressedData.seek(blockStartOffset);
+        
+        numDocsInBlock = compressedData.readVInt();
+        assert numDocsInBlock <= Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+        uncompressedDocEnds = new int[numDocsInBlock];
+        uncompressedBlockLength = 0;        
+
+        int onlyLength = -1;
+        for (int i = 0; i < numDocsInBlock; i++) {
+          if (i == 0) {
+            // The first length value is special. It is shifted and has a bit to denote if
+            // all other values are the same length
+            int lengthPlusSameInd = compressedData.readVInt();
+            int sameIndicator = lengthPlusSameInd & 1;
+            int firstValLength = lengthPlusSameInd >>1;
+            if (sameIndicator == 1) {
+              onlyLength = firstValLength;
+            }
+            uncompressedBlockLength += firstValLength;            
+          } else {
+            if (onlyLength == -1) {
+              // Various lengths are stored - read each from disk
+              uncompressedBlockLength += compressedData.readVInt();            
+            } else {
+              // Only one length 
+              uncompressedBlockLength += onlyLength;
+            }
+          }
+          uncompressedDocEnds[i] = uncompressedBlockLength;
 
 Review comment:
   maybe we could call it `uncompressedDocStarts` and set the index at `i+1` which would then help below to remove the else block of the `docInBlockId > 0` condition below?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] juanka588 commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
juanka588 commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r377544478
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      boolean success = false;
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+        success = true;
+      } finally {
+        if (success == false) {
+          IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        }
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        
+        // Optimisation - check if all lengths are same
+        boolean allLengthsSame = true && numDocsInCurrentBlock >0  ;
+        for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK && allLengthsSame; i++) {
+          if (i > 0 && docLengths[i] != docLengths[i-1]) {
+            allLengthsSame = false;
+          }
+        }
+        if (allLengthsSame) {
+            // Only write one value shifted. Steal a bit to indicate all other lengths are the same
+            int onlyOneLength = (docLengths[0] <<1) | 1;
+            data.writeVInt(onlyOneLength);
+        } else {
+          for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) {
+            if(i == 0) {
+              // Write first value shifted and steal a bit to indicate other lengths are to follow
+              int multipleLengths = (docLengths[0] <<1);
+              data.writeVInt(multipleLengths);              
+            } else {
+              data.writeVInt(docLengths[i]);
+            }
+          }
+        }
+        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
+        LZ4.compress(block, 0, uncompressedBlockLength, data, ht);
+        numDocsInCurrentBlock = 0;
+        // Ensure initialized with zeroes because full array is always written
+        Arrays.fill(docLengths, 0);
+        uncompressedBlockLength = 0;
+        maxPointer = data.getFilePointer();
+        tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
+      }
+    }
+    
+    void writeMetaData() throws IOException {
+      if (blockAddressesStart < 0 ) {
+        return;
+      }
+      
+      long startDMW = data.getFilePointer();
+      meta.writeLong(startDMW);
+      
+      meta.writeVInt(totalChunks);
+      meta.writeVInt(Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK);
+      meta.writeVInt(maxUncompressedBlockLength);
       meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+      
+    
+      CodecUtil.writeFooter(tempBinaryOffsets);
+      IOUtils.close(tempBinaryOffsets);             
+      //write the compressed block offsets info to the meta file by reading from temp file
+      try (ChecksumIndexInput filePointersIn = state.directory.openChecksumInput(tempBinaryOffsets.getName(), IOContext.READONCE)) {
+        CodecUtil.checkHeader(filePointersIn, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT,
+          Lucene80DocValuesFormat.VERSION_CURRENT);
+        Throwable priorE = null;
+        try {
+          final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance(meta, data, totalChunks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+          long fp = blockAddressesStart;
+          for (int i = 0; i < totalChunks; ++i) {
+            filePointers.add(fp);
+            fp += filePointersIn.readVLong();
+          }
+          if (maxPointer < fp) {
+            throw new CorruptIndexException("File pointers don't add up ("+fp+" vs expected "+maxPointer+")", filePointersIn);
+          }
+          filePointers.finish();
+        } catch (Throwable e) {
+          priorE = e;
+        } finally {
+          CodecUtil.checkFooter(filePointersIn, priorE);
+        }
+      }
+      // Write the length of the DMW block in the data 
+      meta.writeLong(data.getFilePointer() - startDMW);
+    }
 
-      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
-      long addr = 0;
-      writer.add(addr);
-      values = valuesProducer.getBinary(field);
+    @Override
+    public void close() throws IOException {
+      if (tempBinaryOffsets != null) {
+        IOUtils.close(tempBinaryOffsets);             
+        state.directory.deleteFile(tempBinaryOffsets.getName());
+        tempBinaryOffsets = null;
+      }
+    }
+    
+  }
+  
+
+  @Override
+  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene80DocValuesFormat.BINARY);
+
+    try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()){
+      BinaryDocValues values = valuesProducer.getBinary(field);
+      long start = data.getFilePointer();
+      meta.writeLong(start); // dataOffset
 
 Review comment:
   I think we should use the BinaryEntry object here, and the just make the object "Writable" to a given DataOutput and "Readable" from a DataInput (which is already the case: readBinaryEntry). This will avoid the comments in the code -2 == docsWithFieldOffset etc.

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379299380
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int [] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte [] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+      } catch (Throwable exception) {
+        IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        throw exception;
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        data.writeVInt(numDocsInCurrentBlock);
+        for (int i = 0; i < numDocsInCurrentBlock; i++) {
+          data.writeVInt(docLengths[i]);
+        }
+        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
+        LZ4.compress(block,  0, uncompressedBlockLength, data, ht);
+        numDocsInCurrentBlock = 0;
+        uncompressedBlockLength = 0;
+        maxPointer = data.getFilePointer();
+        tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
+      }
+    }
+    
+    void writeMetaData() throws IOException {
+      if (blockAddressesStart < 0 ) {
 
 Review comment:
   This makes sense, can you leave a comment about it?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r374744879
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,168 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter  implements Closeable {
 
 Review comment:
   Extra space character before `implements`?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov edited a comment on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
msokolov edited a comment on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583519622
 
 
   > The LZ4 compressed versions of this content were only marginally bigger than their raw counterparts 
   
   Did you also test read performance in this incompressible case?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379304074
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -59,18 +60,18 @@
   private long ramBytesUsed;
   private final IndexInput data;
   private final int maxDoc;
+  private int version = -1;
 
   /** expert: instantiates a new reader */
   Lucene80DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     this.maxDoc = state.segmentInfo.maxDoc();
     ramBytesUsed = RamUsageEstimator.shallowSizeOfInstance(getClass());
 
-    int version = -1;
 
 Review comment:
   maybe keep this variable actually, it would help make version final by doing `this.version = version;` after the try block?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov commented on issue #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
msokolov commented on issue #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#issuecomment-583519622
 
 
   > The LZ4 compressed versions of this content were only marginally bigger than their raw counterparts 
   Did you also test read performance in this incompressible case?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r376531952
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,131 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
 
 Review comment:
   in the past we've put these constants in the meta file and BinaryEntry so that it's easier to change values over time

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r374746978
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
 ##########
 @@ -742,6 +755,107 @@ public BytesRef binaryValue() throws IOException {
         };
       }
     }
+  }  
+  
+  // Decompresses blocks of binary values to retrieve content
+  class BinaryDecoder {
+    
+    private final LongValues addresses;
+    private final IndexInput compressedData;
+    // Cache of last uncompressed block 
+    private long lastBlockId = -1;
+    private int []uncompressedDocEnds = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
+    private int uncompressedBlockLength = 0;        
+    private int numDocsInBlock = 0;
+    private final byte[] uncompressedBlock;
+    private BytesRef uncompressedBytesRef;
+    
+    public BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize) {
+      super();
+      this.addresses = addresses;
+      this.compressedData = compressedData;
+      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
+      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
+      
+    }
+
+    BytesRef decode(int docNumber) throws IOException {
+      int blockId = docNumber >> Lucene80DocValuesFormat.BINARY_BLOCK_SHIFT; 
+      int docInBlockId = docNumber % Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      assert docInBlockId < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK;
+      
+      
+      // already read and uncompressed?
+      if ( blockId != lastBlockId) {
 
 Review comment:
   Remove space after `(` before `blockId`?

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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1234: Add compression for Binary doc value fields
URL: https://github.com/apache/lucene-solr/pull/1234#discussion_r379306909
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java
 ##########
 @@ -353,67 +360,193 @@ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutp
     }
   }
 
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeInt(field.number);
-    meta.writeByte(Lucene80DocValuesFormat.BINARY);
-
-    BinaryDocValues values = valuesProducer.getBinary(field);
-    long start = data.getFilePointer();
-    meta.writeLong(start); // dataOffset
-    int numDocsWithField = 0;
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = 0;
-    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-      numDocsWithField++;
-      BytesRef v = values.binaryValue();
-      int length = v.length;
-      data.writeBytes(v.bytes, v.offset, v.length);
-      minLength = Math.min(length, minLength);
-      maxLength = Math.max(length, maxLength);
+  class CompressedBinaryBlockWriter implements Closeable {
+    FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();    
+    int uncompressedBlockLength = 0;
+    int maxUncompressedBlockLength = 0;
+    int numDocsInCurrentBlock = 0;
+    int[] docLengths = new int[Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; 
+    byte[] block = new byte [1024 * 16];
+    int totalChunks = 0;
+    long maxPointer = 0;
+    long blockAddressesStart = -1; 
+
+    private IndexOutput tempBinaryOffsets;
+    
+    
+    public CompressedBinaryBlockWriter() throws IOException {
+      tempBinaryOffsets = state.directory.createTempOutput(state.segmentInfo.name, "binary_pointers", state.context);
+      boolean success = false;
+      try {
+        CodecUtil.writeHeader(tempBinaryOffsets, Lucene80DocValuesFormat.META_CODEC + "FilePointers", Lucene80DocValuesFormat.VERSION_CURRENT);
+        success = true;
+      } finally {
+        if (success == false) {
+          IOUtils.closeWhileHandlingException(this); //self-close because constructor caller can't 
+        }
+      }
     }
-    assert numDocsWithField <= maxDoc;
-    meta.writeLong(data.getFilePointer() - start); // dataLength
 
-    if (numDocsWithField == 0) {
-      meta.writeLong(-2); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else if (numDocsWithField == maxDoc) {
-      meta.writeLong(-1); // docsWithFieldOffset
-      meta.writeLong(0L); // docsWithFieldLength
-      meta.writeShort((short) -1); // jumpTableEntryCount
-      meta.writeByte((byte) -1);   // denseRankPower
-    } else {
-      long offset = data.getFilePointer();
-      meta.writeLong(offset); // docsWithFieldOffset
-      values = valuesProducer.getBinary(field);
-      final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-      meta.writeShort(jumpTableEntryCount);
-      meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
+    void addDoc(int doc, BytesRef v) throws IOException {
+      if (blockAddressesStart < 0) {
+        blockAddressesStart = data.getFilePointer();
+      }
+      docLengths[numDocsInCurrentBlock] = v.length;
+      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
+      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
+      uncompressedBlockLength += v.length;
+      numDocsInCurrentBlock++;
+      if (numDocsInCurrentBlock == Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
+        flushData();
+      }      
     }
 
-    meta.writeInt(numDocsWithField);
-    meta.writeInt(minLength);
-    meta.writeInt(maxLength);
-    if (maxLength > minLength) {
-      start = data.getFilePointer();
-      meta.writeLong(start);
+    private void flushData() throws IOException {
+      if (numDocsInCurrentBlock > 0) {
+        // Write offset to this block to temporary offsets file
+        totalChunks++;
+        long thisBlockStartPointer = data.getFilePointer();
+        
+        // Optimisation - check if all lengths are same
+        boolean allLengthsSame = true && numDocsInCurrentBlock >0  ;
+        for (int i = 0; i < Lucene80DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK && allLengthsSame; i++) {
 
 Review comment:
   in general we do a `break` when setting `allLengthsSame = false` instead of adding it to the exit condition of the for statement

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


With regards,
Apache Git Services

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