antonha commented on PR #13149:
URL: https://github.com/apache/lucene/pull/13149#issuecomment-2016941949

   > Before merging I'd be curious to better understand why the JVM doesn't 
optimize this better. Presumably, it should be able to resolve the virtual call 
once for the entire for loop rather than doing it again on every iteration? I 
wonder if there is actually a performance bug, or if we are just insufficiently 
warming up the JVM and this for loop never gets compiled through C2?
   
   Valid questions. 
   
   I ran the same luceneutils benchmark with `-XX:+LogCompilation` and 
`-XX:PrintInlining` and I can see c2/level4 compilations being done, including 
`BKDPointTree::addAll` and `DocIdsWriter::readInts32`. The Elasticsearch 
benchmarks mentioned in my post (where we saw similar results) were also run 
long enough that I would be _very_ surprised if c2 had not kicked in. 
   
   I also tried reproducing this behavior in a shorter java program for 
demonstration purposes - with virtual calls similar to the `IntersectVisitor`. 
The program has 3 implementations of an interface processing `int` values, with 
both single value and batch implementations:
   
   ```java
   import java.util.Random;
   
   public class C2Inlining {
     static int ITERATIONS = 50_000;
     static int NUM_VALUES = 100_000;
     public static void main(String[] args) {
       //Generate ints
       Random r = new Random();
       int[] arr = new int[NUM_VALUES];
       for (int i = 0; i < NUM_VALUES; i++) {
         arr[i] = r.nextInt();
       }
   
       IntProcessor[] intProcessors = {
           new IntProcessor1(),
           new IntProcessor2(),
           //Comment this last one out to trigger bimorphic behaviour
           new IntProcessor3()
       };
       processOneByOne(intProcessors, arr);
       processBatch(intProcessors, arr);
     }
   
     private static void processOneByOne(IntProcessor[] intProcessors, int[] 
arr) {
       long start = System.nanoTime();
       for (int i = 0; i < ITERATIONS; i++) {
         for (IntProcessor intProcessor : intProcessors) {
           for (int value : arr) {
             intProcessor.process(value);
           }
         }
       }
       long end = System.nanoTime();
       long took = end - start;
       System.out.printf("One-by-one: Time per iteration: %.3f ms%n", 
(((double) took) / ITERATIONS) / 1_000_000d);
     }
   
     private static void processBatch(IntProcessor[] intProcessors, int[] arr) {
       long start = System.nanoTime();
       for (int i = 0; i < ITERATIONS; i++) {
         for (IntProcessor intProcessor : intProcessors) {
           intProcessor.process(arr);
         }
       }
       long end = System.nanoTime();
       long took = end - start;
       System.out.printf("Batch: Time per iteration: %.3f ms%n", (((double) 
took) / ITERATIONS) / 1_000_000d);
     }
   
     interface IntProcessor {
       void process(int i);
       void process(int[] arr);
     }
   
     static class IntProcessor1 implements IntProcessor {
   
       static int value;
   
       @Override
       public void process(int i) {
         value = i;
       }
   
       @Override
       public void process(int[] arr) {
         for (int i = 0; i < arr.length; i++) {
           value = arr[i];
         }
       }
     }
   
     static class IntProcessor2 implements IntProcessor {
       static int value;
   
       @Override
       public void process(int i) {
         value = i;
       }
   
       @Override
       public void process(int[] arr) {
         for (int i = 0; i < arr.length; i++) {
           value = arr[i];
         }
       }
     }
   
     static class IntProcessor3 implements IntProcessor {
       static int value;
   
       @Override
       public void process(int i) {
         value = i;
       }
   
       @Override
       public void process(int[] arr) {
         for (int i = 0; i < arr.length; i++) {
           value = arr[i];
         }
       }
     }
   }
   
   ```
   
   I ran this in this form, and with one implementation commented out (to 
trigger the bimorphic inlining behavior). The timing results are quite extreme 
(running with jdk21 and `-server`)
   
   | Variant | One-by-One | Batch |
   |--- | --- | --- |
   | **Three implementations** | 1.101 ms/iter | 0.018 ms/iter | 
   | **Two implementations** |  0.198 ms/iter | 0.012 ms/iter | 
   
   I.e. with three implementations, batching is ~60 times faster than single 
implementations. With two, it is ~16 times faster. 
   Both versions saw c2 compilations. 
   
   This is the same pattern that we see with the lucene code - although the 
difference is much more extreme (I'm guessing due to the implementation). I do 
think that we can draw the conclusion that helping the JVM with batch versions 
of virtual calls can help performance significantly. 
   
   One might argue that the JVM should be able to figure this out on it's own. 
But until it does, let's maybe help it a bit?


-- 
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-unsubscr...@lucene.apache.org

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


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

Reply via email to