You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/09/29 15:20:17 UTC

[GitHub] [lucene] jpountz opened a new pull request #327: LUCENE-10125: Speed up DirectWriter.

jpountz opened a new pull request #327:
URL: https://github.com/apache/lucene/pull/327


   There was a regression introduced in
   https://github.com/apache/lucene/pull/107/files#diff-49b11ced76acedf749c5a5a0ff6e7fe93b8fb64caf8697e487a56f4f7adbb510
   where we moved from write logic that was optimized for every number of bits per
   value to more general logic that had to work for every number of bits per value.
   
   This PR doesn't restore as much specialization, but some middle ground that
   makes flushes and merges of doc values noticeably faster (though not much
   faster).


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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719166524



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       @jpountz The idea with a functional interface like in #333 was also my idea how to simplify DirectReader, which has code duplication. All DirectReaderXY classes have exact same copypasted code, only the get method is different.
   
   My idea would be to only hae a single and final DirectReader class which gets instantiated with a private functional interface to implement get(). I just had no time, but this would simplify code a bit. Instead of those many inner classes, you would just have many static methods "get12()", "get48()" and use those with a method reference.




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719106766



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       Thanks Uwe, the exception handing is a bit annoying indeed. Have you been able to check whether either approach makes any performance difference?




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719130968



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       > the exception handing is a bit annoying indeed. 
   
   The problem is that you cant do much with adapting VarHandles like applying an explicit cast. To do this you need to convert it to a MethodHandle (which is internally identical to the VarHandle).
   
   BTW, initially the VarHandles are converted to MethodHandles in the JDK code until they are optimized away by C2. But the Exceptions are ignored (MethodHandles are only called from generated bytecode and the exception handling is simply left out, like in painless).
   
   In JDK 17's Project Panama they added the missing adaptors to do casts: https://docs.oracle.com/en/java/javase/17/docs/api/jdk.incubator.foreign/jdk/incubator/foreign/MemoryHandles.html#asUnsigned(java.lang.invoke.VarHandle,java.lang.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.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719401059



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       After the bad experiments in #335 I am not really confident that this brings too much, maybe it's worse.
   
   I would start with benchmarking the #333 one. If this one behaves bad, we can still try #332 or leave everything as 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.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719166524



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       @jpountz The idea with a functional interface like in #333 was also my idea how to simplify DirectReader, which has code duplication. All DirectReaderXY classes have exact same copypasted code, only the get method is different. I looked at this a few months ago and then forget about it. 
   
   My idea would be to only hae a single and final DirectReader class which gets instantiated with a private functional interface to implement get(). I just had no time, but this would simplify code a bit. Instead of those many inner classes, you would just have many static methods "get12()", "get48()" and use those with a method reference.




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r718852626



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       Wouldn't it be better to have the if statement outside of loop? So have a separate loop per bit size?
   
   In addition we could use a casting adapter using a MethodHandle on top of the VarHandles. This would allow hotspot to have one method signature taking byte array, offset and a long, while casting is done inside the MethodHandle. This would make the signature identical and you could just switch handle but code stays.




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] dweiss commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
dweiss commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719102651



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       bq. I was hoping that the compiler would automatically do this since bitsPerValuedoesn't change across iterations.
   
   It should hoist it as a loop 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.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r718919001



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       I drafted #332. Not really happy, but worth a try. I wasnt able to do perf testing...
   
   The loops now look really clean, backsides are: MethodHandles are declared to throw Throwable (who invented this shit?!?)




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719130078



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       I'm off for a long weekend but I can run benchmarks on Monday.
   
   > how many iterations have the loop?
   
   These loops have 64 or 128 iterations.




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719127442



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       Hi,
   it was too late yesterday to do any test. I just drafted my idea and went to sleep.
   
   I don't know which benchmark you used (lucenebench and how was it called - the taxidriver bench was completely new for me; I have no idea how to start it?) I can quickly start a benchmark but I want numbers comparable so it should be same than the one you used.
   
   > It should hoist it as a loop constant.
   
   I hope so, I am just afraid that the code is too complex (4 branches). In addition: how many iterations have the loop? If it is called often for loops with only few elements, then I am not sure if it helps, because whenever the bitsize changes when encode() is called, it will be deoptimized and it starts over again.
   
   Because of this I made the 2 drafts. I would now prefer the #333 because of cleaner code. Both should behave similar.




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r718867031



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       I was hoping that the compiler would automatically do this since `bitsPerValue`doesn't change across iterations. But please open a PR, I'm curious to have a look at the MethodHandle approach. :)




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r718852626



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       Wouldn't it be better to have the if statement outside of loop? So have a separate loop per bit size?
   
   In addition we could use a casting adapter using a MethodHandle on top of the VarHandles. This would allow hotspot to have one method signature taking byte array, offset and a long, while casting is done inside the MethodHandle. This would make the signature identical and you could just switch handle but code stays.

##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       The casting adapter would help to assign the correct VarHandles outside the loop and then have a simple loop without branches.
   
   Should I give it a try in a separate PR, because this is already committed?

##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       I drafted #332. Not really happy, but worth a try. I wasnt able to do perf testing...
   
   The loops now look really clean, backsides are: MethodHandles are declared to throw Throwable (who invented this shit?!?)

##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       I drafted another one #333. This one is using a simple functional interface. It does basically the same like the #332, but without shitty exception handling.




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r718856191



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       The casting adapter would help to assign the correct VarHandles outside the loop and then have a simple loop without branches.
   
   Should I give it a try in a separate PR, because this is already committed?




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719401321



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       Thanks in any case for trying out!




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r718935962



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       I drafted another one #333. This one is using a simple functional interface. It does basically the same like the #332, but without shitty exception handling.




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz merged pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
jpountz merged pull request #327:
URL: https://github.com/apache/lucene/pull/327


   


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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719199226



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       Here is the other issue / PR #335.




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
uschindler commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r719130968



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       > the exception handing is a bit annoying indeed. 
   
   The problem is that you cant do much with adapting VarHandles like applying an explicit cast. To do this you need to convert it to a MethodHandle (which is internally identical to the VarHandle).
   
   BTW, initially the VarHandles are converted to MethodHandles in the JDK code until they are optimized away by C2. But the Exceptions are ignored (MethodHandles are only called from generated bytecode and the exception handling is simply left out, like in painless).




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on a change in pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #327:
URL: https://github.com/apache/lucene/pull/327#discussion_r718867031



##########
File path: lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
##########
@@ -94,38 +91,54 @@ private void flush() throws IOException {
     }
     // Avoid writing bits from values that are outside of the range we need to encode
     Arrays.fill(nextValues, off, nextValues.length, 0L);
-    encode(nextValues, 0, nextBlocks, 0, iterations);
+    encode(nextValues, off, nextBlocks, bitsPerValue);
     final int blockCount =
         (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsPerValue);
     output.writeBytes(nextBlocks, blockCount);
     off = 0;
   }
 
-  public void encode(
-      long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
-    int nextBlock = 0;
-    int bitsUsed = 0;
-    for (int i = 0; i < byteValueCount * iterations; ++i) {
-      final long v = values[valuesOffset++];
-      assert PackedInts.unsignedBitsRequired(v) <= bitsPerValue;
-      if (bitsUsed < byteOffset) {
-        // just buffer
-        nextBlock |= v << bitsUsed;
-        bitsUsed += bitsPerValue;
-      } else {
-        // flush as many blocks as possible
-        blocks[blocksOffset++] = (byte) (nextBlock | (v << bitsUsed));
-        int bits = 8 - bitsUsed;
-        while (bits <= bitsUsedOffset) {
-          blocks[blocksOffset++] = (byte) (v >> bits);
-          bits += 8;
+  private static void encode(long[] nextValues, int upTo, byte[] nextBlocks, int bitsPerValue) {
+    if ((bitsPerValue & 7) == 0) {
+      // bitsPerValue is a multiple of 8: 8, 16, 24, 32, 30, 48, 56, 64
+      final int bytesPerValue = bitsPerValue / Byte.SIZE;
+      for (int i = 0, o = 0; i < upTo; ++i, o += bytesPerValue) {

Review comment:
       I was hoping that the compiler would automatically do this since `bitsPerValue`doesn't change across iterations. But please open a PR, I'm curious to have a look at the MethodHandle approach. :)




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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz merged pull request #327: LUCENE-10125: Speed up DirectWriter.

Posted by GitBox <gi...@apache.org>.
jpountz merged pull request #327:
URL: https://github.com/apache/lucene/pull/327


   


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

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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



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