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/06/08 11:13:03 UTC

[GitHub] [lucene] uschindler opened a new pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-25)

uschindler opened a new pull request #177:
URL: https://github.com/apache/lucene/pull/177


   **INFO: This is a followup of #173: It's the same code base, but with API changes from JDK 17 applied**
   
   This is just a draft PR for a first insight on memory mapping improvements in JDK 17+.
   
   Some background information: Starting with JDK-14, there is a new incubating module "jdk.incubator.foreign" that has a new, not yet stable API for accessing off-heap memory (and later it will also support calling functions using classical MethodHandles that are located in libraries like .so or .dll files). This incubator module has several versions:
   - first version: https://openjdk.java.net/jeps/370 (slow, very buggy and thread confinement, so making it unuseable with Lucene)
   - second version: https://openjdk.java.net/jeps/383 (still thread confinement, but now allows transfer of "ownership" to other threads; this is still impossible to use with Lucene.
   - third version in JDK 16: https://openjdk.java.net/jeps/393 (this version has included "Support for shared segments"). This now allows us to safely use the same external mmaped memory from different threads and also unmap it! This was implemented in the previous pull request #173
   - fourth version in JDK 17: included in build 25. This mainly changes the API around the scopes. Instead of having segments explicitely made "shared", we can assign them to some resource scope which control their behaviour. The resourceScope is produced one time for each IndexInput instance (not clones) and owns all segments. When the resourceScope is closed, all segments get invalid - and we throw `AlreadyClosedException`.
   
   This module more or less overcomes several problems:
   - ByteBuffer API is limited to 32bit (in fact MMapDirectory has to chunk in 1 GiB portions)
   - There is no official way to unmap ByteBuffers when the file is no longer used. There is a way to use `sun.misc.Unsafe` and forcefully unmap segments, but any IndexInput accessing the file from another thread will crush the JVM with SIGSEGV or SIGBUS. We learned to live with that and we happily apply the unsafe unmapping, but that's the main issue.
   
   @uschindler had many discussions with the team at OpenJDK and finally with the third incubator, we have an API that works with Lucene. It was very fruitful discussions (thanks to @mcimadamore !)
   
   With the third incubator we are now finally able to do some tests (especially performance). As this is an incubating module, this PR first changes a bit the build system:
   - disable `-Werror` for `:lucene:core`
   - add the incubating module to compiler of `:lucene:core` and enable it for all test builds. This is important, as you have to pass `--add-modules jdk.incubator.foreign` also at runtime!
   
   The code basically just modifies `MMapDirectory` to use LONG instead of INT for the chunk size parameter. In addition it adds `MemorySegmentIndexInput` that is a copy of our `ByteBufferIndexInput` (still there, but unused), but using MemorySegment instead of ByteBuffer behind the scenes. It works in exactly the same way, just the try/catch blocks for supporting EOFException or moving to another segment were rewritten.
   
   It passes all tests and it looks like you can use it to read indexes. The default chunk size is now 16 GiB (but you can raise or lower it as you like; tests are doing this). Of course you can set it to Long.MAX_VALUE, in that case every index file is always mapped to one big memory mapping. My testing with Windows 10 have shown, that this is *not a good idea!!!*. Huge mappings fragment address space over time and as we can only use like 43 or 46 bits (depending on OS), the fragmentation will at some point kill you. So 16 GiB looks like a good compromise: Most files will be smaller than 6 GiB anyways (unless you optimize your index to one huge segment). So for most Lucene installations, the number of segments will equal the number of open files, so Elasticsearch huge user consumers will be very happy. The sysctl max_map_count may not need to be touched anymore.
   
   In addition, this implements `readLongs` in a better way than @jpountz did (no caching or arbitrary objects). Nevertheless, as the new MemorySegment API relies on final, unmodifiable classes and coping memory from a MemorySegment to a on-heap Java array, it requires us to wrap all those arrays using a MemorySegment each time (e.g. in `readBytes()` or `readLELongs`), there may be some overhead du to short living object allocations (those are NOT reuseable!!!). _In short: In future we should throw away on coping/loading our stuff to heap and maybe throw away IndexInput completely and base our code fully on random access. The new foreign-vector APIs will in future also be written with MemorySegment in its focus. So you can allocate a vector view on a MemorySegment and let the vectorizer fully work outside java heap inside our mmapped files! :-)_
   
   It would be good if you could checkout this branch and try it in production.
   
   But be aware:
   - You need JDK 11 to run Gradle (set `JAVA_HOME` to it)
   - You need JDK 17-ea-b25 (set `RUNTIME_JAVA_HOME` to it)
   - The lucene-core.jar will be JDK17 class files and requires JDK-17 to execute.
   - Also you need to add `--add-modules jdk.incubator.foreign` to the command line of your Java program/Solr server/Elasticsearch server
   
   It would be good to get some benchmarks, especially by @rmuir or @mikemccand. _Take your time and enjoy the complexity of setting this up!_ ;-)
   
   My plan is the following:
   - report any bugs or slowness, especially with Hotspot optimizations. The last time I talked to Maurizio, he taked about Hotspot not being able to fully optimize for-loops with long instead of int, so it may take some time until the full performance is there.
   - wait until the final version of project PANAMA-foreign goes into Java's Core Library (no module needed anymore)
   - add a MR-JAR for lucene-core.jar and compile the MemorySegmentIndexInput and maybe some helper classes with JDK 18/19 (hopefully?).


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

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



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


[GitHub] [lucene] markrmiller commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
markrmiller commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-935338177


   Just stumbled on this. I've also recently thought a bit about Lucene JMH benchmarks that are semi equivalent to lucene-util benchmarks.  I've been spending some free time here and there trying to bring some JMH to lucene-util, mainly perfasm output and async profiler, for which I also have a rough patch to use in the tests waiting for some clean up. I have some rough working stuff, but there tends to be a lot of resulting noise in the outputs that I have to look at filtering out or targeting the capture more with a more invasive integration. More than once, I've thought, it would be nicer to just offer the same suite of benchmarks within JMH and capture the other things it offers as well. 
   
   As Robert says, it's no replacement for realistic high level benchmarks in most case, especially for Solr, I've used the same unit/integration test analogy, but it's fantastic for fast, reliable feedback with superb introspection tools built in.
   
   Also, those lambdas *are* damn slow until hotspot takes care of them :) I saw that slowness way back and chased what the deal was for a while, I was ready to toss them out of my toolbox, and yeah, they just need time. Then they are fine, probably not great for places that don't run a lot.
   
   JMH will actually run the profilers during warmup runs to warm them up, but then it dumps the data, so it's nice that even profiling just captures the warm iterations if you'd like. I don't like that it also includes setup, as it says, to be sure and catch the edges. I could do without that when you are constructing indexes or something to be queried, but if you setup larger stuff like lucene-util does to have at hand already, that circumvents that.


-- 
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 pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866013290


   There is no fix for the JDK 17 issues with garbage collection and disabled tiered compilation. From Java 18 on we can use the new MemoryCopy class to do bulk copies. It also supports byte swap to adjust endianness: https://github.com/openjdk/panama-foreign/pull/555
   
   The results look promising. I have a branch using that available already, will create a new pull request to record the 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.

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 edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891211684


   Hi,
   
   > > Ca you post `java -version`and tell us if you maybe compiledyour own version?
   > Hi @uschindler,
   > java --version
   > openjdk 17-ea 2021-09-14
   > OpenJDK Runtime Environment (build 17-ea+22-1964)
   > OpenJDK 64-Bit Server VM (build 17-ea+22-1964, mixed mode, sharing)
   
   Please read the title and description of the pull request. The minimum version required is build 25, yours is 22.


-- 
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 #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {
+      return alreadyClosed();
+    }
+    // TODO: maybe open a JDK issue to have a separate, more
+    // meaningful exception for unmapped segments:
+    if (e.getMessage() != null && e.getMessage().contains("closed")) {
+      return alreadyClosed();
+    }
+    return e;
+  }
+
+  RuntimeException handlePositionalIOOBE(String action, long pos) throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position: " + this);
+    } else {
+      throw new EOFException(action + " past EOF: " + this);
+    }
+  }
+
+  private AlreadyClosedException alreadyClosed() {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      do {
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+      } while (curSegment.byteSize() == 0L);
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    final MemorySegment arraySegment = MemorySegment.ofArray(b);
+    try {
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(arraySegment, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  private void readBytesBoundary(final MemorySegment arraySegment, int offset, int len)
+      throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, curAvail));
+        len -= curAvail;
+        offset += curAvail;
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+        curAvail = curSegment.byteSize();
+      }
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public void readLongs(long[] dst, int offset, int length) throws IOException {
+    if (IS_LITTLE_ENDIAN) {
+      final MemorySegment targetSlice =
+          MemorySegment.ofArray(dst).asSlice((long) offset << 3, (long) length << 3);
+      try {
+        targetSlice.copyFrom(curSegment.asSlice(curPosition, targetSlice.byteSize()));
+        curPosition += targetSlice.byteSize();
+      } catch (
+          @SuppressWarnings("unused")
+          IndexOutOfBoundsException iobe) {
+        super.readLongs(dst, offset, length);

Review comment:
       If you are afraid that this is not tested. Don't worry: The MultiMMap test case creates MMapDirectory instances with very small chunk sizes (small as a segments of 32 bytes) and tests all methods - the test code is inherited from ByteBuffer MMAP. Any problem in the catch code would break test :-)




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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861222097


   Here is the results of the original pull request without the unsafe memory copy, using the `fromArray()` and slicing code. During runtime, the benchmark runs 72 seconds vs 60 seconds. The 12 seconds are mostly caused by garbage collection and allocations.
   
   At the end of the report you see an overview of the heap allocations seen by JFR:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
      BrowseMonthTaxoFacets        1.01      (5.3%)        0.71      (5.9%)  -29.7% ( -38% -  -19%) 0.000
       BrowseDateTaxoFacets        0.98      (7.0%)        0.70      (5.9%)  -28.7% ( -38% -  -17%) 0.000
   BrowseDayOfYearTaxoFacets        0.98      (7.0%)        0.70      (5.9%)  -28.7% ( -38% -  -16%) 0.000
                   Wildcard       19.36      (9.9%)       16.30      (6.8%)  -15.8% ( -29% -    0%) 0.000
                 AndHighLow      273.88      (3.5%)      240.26      (2.4%)  -12.3% ( -17% -   -6%) 0.000
            LowSloppyPhrase        9.80      (4.1%)        8.85      (4.3%)   -9.7% ( -17% -   -1%) 0.000
            MedSloppyPhrase        6.82      (5.2%)        6.24      (5.3%)   -8.4% ( -18% -    2%) 0.000
                  OrHighLow      220.56      (3.2%)      202.29      (3.1%)   -8.3% ( -14% -   -2%) 0.000
                    Respell       54.65      (1.4%)       50.44      (1.5%)   -7.7% ( -10% -   -4%) 0.000
           HighSloppyPhrase        3.57      (4.0%)        3.31      (4.8%)   -7.3% ( -15% -    1%) 0.000
                 AndHighMed      116.21      (4.8%)      107.99      (3.6%)   -7.1% ( -14% -    1%) 0.000
                   PKLookup      188.31      (2.6%)      175.71      (2.3%)   -6.7% ( -11% -   -1%) 0.000
                AndHighHigh       26.44      (4.9%)       24.69      (4.2%)   -6.6% ( -15% -    2%) 0.000
                LowSpanNear       26.13      (1.9%)       24.55      (2.5%)   -6.1% ( -10% -   -1%) 0.000
                  OrHighMed       58.52      (4.3%)       55.04      (4.0%)   -5.9% ( -13% -    2%) 0.000
                MedSpanNear       16.21      (2.1%)       15.27      (3.6%)   -5.8% ( -11% -    0%) 0.000
                     Fuzzy2       62.90     (10.2%)       59.78     (13.1%)   -5.0% ( -25% -   20%) 0.182
                     Fuzzy1       47.53      (5.7%)       45.81      (4.7%)   -3.6% ( -13% -    7%) 0.027
      BrowseMonthSSDVFacets        4.13      (5.4%)        3.99      (6.5%)   -3.3% ( -14% -    9%) 0.079
                 TermDTSort       62.99     (15.3%)       60.93     (13.8%)   -3.3% ( -28% -   30%) 0.476
   BrowseDayOfYearSSDVFacets        4.02      (4.2%)        3.89      (5.5%)   -3.2% ( -12% -    6%) 0.036
                 OrHighHigh       17.99      (5.2%)       17.41      (5.4%)   -3.2% ( -13% -    7%) 0.055
                 HighPhrase       15.57      (3.6%)       15.08      (3.0%)   -3.1% (  -9% -    3%) 0.003
       HighTermTitleBDVSort       58.97     (19.8%)       57.13     (21.8%)   -3.1% ( -37% -   48%) 0.636
               HighSpanNear        4.84      (2.1%)        4.69      (1.5%)   -3.1% (  -6% -    0%) 0.000
                  LowPhrase        6.95      (4.7%)        6.76      (3.1%)   -2.7% ( -10% -    5%) 0.031
      HighTermDayOfYearSort       79.36     (13.6%)       77.50     (16.1%)   -2.3% ( -28% -   31%) 0.618
       HighIntervalsOrdered        6.10      (3.7%)        5.96      (4.5%)   -2.3% ( -10% -    6%) 0.079
                  MedPhrase       20.71      (3.9%)       20.50      (2.6%)   -1.0% (  -7% -    5%) 0.326
          HighTermMonthSort       73.31     (12.2%)       72.61     (15.6%)   -1.0% ( -25% -   30%) 0.828
               OrNotHighLow      659.29      (3.6%)      661.62      (4.6%)    0.4% (  -7% -    8%) 0.785
                    Prefix3       56.18      (6.0%)       56.64      (7.3%)    0.8% ( -11% -   15%) 0.699
              OrNotHighHigh      517.12      (4.0%)      525.07      (3.9%)    1.5% (  -6% -    9%) 0.217
              OrHighNotHigh      588.54      (4.1%)      612.06      (4.6%)    4.0% (  -4% -   13%) 0.004
                    LowTerm     1337.81      (3.5%)     1404.26      (3.4%)    5.0% (  -1% -   12%) 0.000
               OrHighNotMed      738.42      (3.2%)      787.11      (4.6%)    6.6% (  -1% -   14%) 0.000
               OrNotHighMed      741.56      (5.8%)      790.89      (4.0%)    6.7% (  -2% -   17%) 0.000
                    MedTerm     1707.00      (3.8%)     1831.49      (4.5%)    7.3% (   0% -   16%) 0.000
               OrHighNotLow      843.88      (4.5%)      906.51      (4.6%)    7.4% (  -1% -   17%) 0.000
                   HighTerm     1929.94      (6.5%)     2137.55      (5.9%)   10.8% (  -1% -   24%) 0.000
                     IntNRQ      268.71      (6.0%)      317.78      (6.2%)   18.3% (   5% -   32%) 0.000
   
   CPU merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_
 version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/bench
 mark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 6.45 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 2155605 events (total: 2M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   10.30%        221932        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.54%         205542        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   5.85%         126158        org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   3.90%         84032         jdk.internal.foreign.AbstractMemorySegmentImpl#<init>()
   3.89%         83875         java.lang.Class#getComponentType()
   3.59%         77332         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   3.43%         74043         jdk.internal.misc.Unsafe#checkPrimitiveArray()
   3.04%         65576         jdk.internal.foreign.AbstractMemorySegmentImpl#checkBounds()
   2.84%         61133         jdk.internal.foreign.AbstractMemorySegmentImpl#checkBoundsSmall()
   2.81%         60625         jdk.internal.misc.ScopedMemoryAccess#getShortUnalignedInternal()
   2.59%         55777         jdk.internal.foreign.MappedMemorySegmentImpl#dup()
   2.50%         53863         org.apache.lucene.store.MemorySegmentIndexInput#readBytes()
   2.36%         50894         jdk.internal.foreign.Utils#filterSegment()
   2.32%         49940         jdk.internal.foreign.HeapMemorySegmentImpl$OfByte#dup()
   2.15%         46330         jdk.internal.foreign.AbstractMemorySegmentImpl#copyFrom()
   1.92%         41393         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   1.65%         35535         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   1.27%         27288         jdk.internal.foreign.HeapMemorySegmentImpl$OfByte#<init>()
   1.20%         25922         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.04%         22431         jdk.internal.foreign.HeapMemorySegmentImpl$OfByte#fromArray()
   0.98%         21203         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.94%         20165         jdk.internal.foreign.SharedScope#checkValidState()
   0.92%         19927         jdk.internal.misc.ScopedMemoryAccess#copyMemoryInternal()
   0.90%         19325         org.apache.lucene.queries.spans.SpanScorer#setFreqCurrentDoc()
   0.85%         18220         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   0.79%         16981         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.69%         14907         org.apache.lucene.search.PhraseScorer$1#matches()
   0.68%         14599         org.apache.lucene.queries.spans.TermSpans#nextStartPosition()
   0.67%         14537         org.apache.lucene.index.SingletonSortedSetDocValues#nextOrd()
   0.67%         14451         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
   
   
   CPU merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-ba
 seline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 4.24 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1752341 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   13.25%        232242        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   8.95%         156849        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   7.49%         131320        java.nio.ByteBuffer#getArray()
   4.91%         85990         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   3.95%         69304         jdk.internal.misc.Unsafe#convEndian()
   3.79%         66477         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   3.31%         57980         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   3.20%         56142         org.apache.lucene.store.ByteBufferGuard#ensureValid()
   2.91%         51021         java.nio.Buffer#scope()
   2.16%         37770         java.nio.ByteBuffer#get()
   2.01%         35217         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   1.71%         29978         org.apache.lucene.store.ByteBufferGuard#getBytes()
   1.70%         29781         java.nio.Buffer#checkIndex()
   1.55%         27212         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.21%         21264         org.apache.lucene.store.ByteBufferIndexInput#readBytes()
   1.21%         21134         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   1.11%         19411         org.apache.lucene.store.ByteBufferGuard#getShort()
   1.09%         19159         org.apache.lucene.queries.spans.SpanScorer#setFreqCurrentDoc()
   1.02%         17794         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.01%         17766         java.nio.Buffer#position()
   0.98%         17108         org.apache.lucene.index.SingletonSortedSetDocValues#nextOrd()
   0.95%         16667         org.apache.lucene.search.PhraseScorer$1#matches()
   0.87%         15251         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.86%         15055         org.apache.lucene.queries.spans.TermSpans#nextStartPosition()
   0.77%         13498         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.75%         13162         org.apache.lucene.store.ByteBufferIndexInput$SingleBufferImpl#seek()
   0.70%         12317         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.68%         11907         org.apache.lucene.codecs.lucene90.ForUtil#expand8()
   0.63%         11117         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$19#ordValue()
   0.62%         10813         org.apache.lucene.util.PriorityQueue#upHeap()
   
   
   HEAP merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified
 _version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/benc
 hmark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 18.34 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 14415681 events (total: 5004408M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   55.11%        2757776M      jdk.internal.foreign.HeapMemorySegmentImpl$OfByte#fromArray()
   22.41%        1121632M      jdk.internal.foreign.HeapMemorySegmentImpl$OfByte#dup()
   20.85%        1043452M      jdk.internal.foreign.MappedMemorySegmentImpl#dup()
   0.71%         35428M        jdk.internal.foreign.HeapMemorySegmentImpl$OfLong#dup()
   0.18%         9206M         jdk.internal.foreign.HeapMemorySegmentImpl$OfLong#fromArray()
   0.12%         6150M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   0.09%         4729M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   0.09%         4620M         org.apache.lucene.util.FixedBitSet#<init>()
   0.06%         3099M         java.util.AbstractList#iterator()
   0.04%         1773M         java.util.ArrayList#grow()
   0.03%         1360M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   0.02%         1090M         java.util.ArrayList#iterator()
   0.02%         1085M         org.apache.lucene.util.PriorityQueue#<init>()
   0.02%         908M          org.apache.lucene.util.ArrayUtil#growExact()
   0.02%         796M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   0.02%         787M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   0.01%         577M          org.apache.lucene.util.BytesRef#<init>()
   0.01%         537M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   0.01%         517M          org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   0.01%         394M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   0.01%         370M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   0.01%         366M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   0.01%         352M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   0.01%         349M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   0.01%         322M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   0.01%         317M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   0.01%         295M          org.apache.lucene.store.MemorySegmentIndexInput#buildSlice()
   0.01%         261M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   0.00%         240M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.00%         237M          java.util.Arrays#copyOfRange()
   
   
   HEAP merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-b
 aseline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 2.55 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 103186 events (total: 37912M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   15.31%        5806M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   12.65%        4796M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   12.47%        4726M         org.apache.lucene.util.FixedBitSet#<init>()
   8.56%         3246M         java.util.AbstractList#iterator()
   5.25%         1990M         java.util.ArrayList#grow()
   3.74%         1419M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   2.92%         1108M         org.apache.lucene.util.PriorityQueue#<init>()
   2.59%         981M          java.util.ArrayList#iterator()
   2.35%         890M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   2.32%         879M          org.apache.lucene.util.ArrayUtil#growExact()
   2.08%         789M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   1.64%         621M          org.apache.lucene.util.BytesRef#<init>()
   1.40%         530M          org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   1.22%         461M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.21%         458M          java.nio.DirectByteBufferR#duplicate()
   1.14%         433M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.03%         388M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.00%         377M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   0.92%         349M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   0.87%         330M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   0.87%         328M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   0.83%         316M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   0.70%         265M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   0.63%         240M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.60%         229M          java.nio.DirectByteBufferR#slice()
   0.52%         198M          org.apache.lucene.store.ByteBufferIndexInput#newCloneInstance()
   0.51%         193M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.48%         180M          java.util.AbstractList#listIterator()
   0.47%         179M          org.apache.lucene.codecs.lucene90.Lucene90ScoreSkipReader#readImpacts()
   0.47%         178M          java.nio.DirectByteBufferR#asLongBuffer()
    


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-856778316


   Linux Jenkins tests: https://jenkins.thetaphi.de/job/Lucene-jdk17panama-Linux/
   Windows Jenkins tests: https://jenkins.thetaphi.de/job/Lucene-jdk17panama-Windows/


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

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



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


[GitHub] [lucene] msokolov commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -231,55 +224,60 @@ public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
     ensureCanRead(name);
     Path path = directory.resolve(name);
-    try (FileChannel c = FileChannel.open(path, StandardOpenOption.READ)) {
-      final String resourceDescription = "MMapIndexInput(path=\"" + path.toString() + "\")";
-      final boolean useUnmap = getUseUnmap();
-      return ByteBufferIndexInput.newInstance(
-          resourceDescription,
-          map(resourceDescription, c, 0, c.size()),
-          c.size(),
-          chunkSizePower,
-          new ByteBufferGuard(resourceDescription, useUnmap ? CLEANER : null));
+    final String resourceDescription = "MemorySegmentIndexInput(path=\"" + path.toString() + "\")";
+    final long fileSize = Files.size(path);
+
+    boolean success = false;
+    final ResourceScope scope = ResourceScope.newSharedScope();
+    try {
+      final MemorySegment[] segments = map(scope, resourceDescription, path, fileSize);
+      final IndexInput in =
+          MemorySegmentIndexInput.newInstance(
+              resourceDescription, scope, segments, fileSize, chunkSizePower);
+      success = true;
+      return in;

Review comment:
       So in this case the `ResourceScope` remains open. Don't we need to close it when closing the `MMapDirectory`?

##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       Should it be `Unwrappable`? Or maybe ... `Wrapped`?

##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {

Review comment:
       scary - why do we have this? Who is throwing NPE? do we have null segments in the array?

##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {
+      return alreadyClosed();
+    }
+    // TODO: maybe open a JDK issue to have a separate, more
+    // meaningful exception for unmapped segments:
+    if (e.getMessage() != null && e.getMessage().contains("closed")) {
+      return alreadyClosed();
+    }
+    return e;
+  }
+
+  RuntimeException handlePositionalIOOBE(String action, long pos) throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position: " + this);
+    } else {
+      throw new EOFException(action + " past EOF: " + this);
+    }
+  }
+
+  private AlreadyClosedException alreadyClosed() {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      do {
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+      } while (curSegment.byteSize() == 0L);
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    final MemorySegment arraySegment = MemorySegment.ofArray(b);
+    try {
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(arraySegment, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  private void readBytesBoundary(final MemorySegment arraySegment, int offset, int len)
+      throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, curAvail));
+        len -= curAvail;
+        offset += curAvail;
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+        curAvail = curSegment.byteSize();
+      }
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));

Review comment:
       Is it OK to call this if len == 0?

##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {
+      return alreadyClosed();
+    }
+    // TODO: maybe open a JDK issue to have a separate, more
+    // meaningful exception for unmapped segments:
+    if (e.getMessage() != null && e.getMessage().contains("closed")) {
+      return alreadyClosed();
+    }
+    return e;
+  }
+
+  RuntimeException handlePositionalIOOBE(String action, long pos) throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position: " + this);
+    } else {
+      throw new EOFException(action + " past EOF: " + this);
+    }
+  }
+
+  private AlreadyClosedException alreadyClosed() {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      do {
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+      } while (curSegment.byteSize() == 0L);
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    final MemorySegment arraySegment = MemorySegment.ofArray(b);
+    try {
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(arraySegment, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  private void readBytesBoundary(final MemorySegment arraySegment, int offset, int len)
+      throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, curAvail));
+        len -= curAvail;
+        offset += curAvail;
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+        curAvail = curSegment.byteSize();
+      }
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public void readLongs(long[] dst, int offset, int length) throws IOException {
+    if (IS_LITTLE_ENDIAN) {
+      final MemorySegment targetSlice =
+          MemorySegment.ofArray(dst).asSlice((long) offset << 3, (long) length << 3);
+      try {
+        targetSlice.copyFrom(curSegment.asSlice(curPosition, targetSlice.byteSize()));
+        curPosition += targetSlice.byteSize();
+      } catch (
+          @SuppressWarnings("unused")
+          IndexOutOfBoundsException iobe) {
+        super.readLongs(dst, offset, length);

Review comment:
       I'm confused about these calls to `super` methods. Is the idea -- we use the MemorySegment methods if the read is within a single segment, otherwise we fall back to one-at-a-time decoding? If so, should we eventually handle multiple bulk reads here, and/or do the bounds-checking arithmetic ourselves rather than rely on handling IOOBE?




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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891211684


   Hi,
   
   > > Ca you post `java -version`and tell us if you maybe compiledyour own version?
   > > Hi @uschindler,
   > > java --version
   > > openjdk 17-ea 2021-09-14
   > > OpenJDK Runtime Environment (build 17-ea+22-1964)
   > > OpenJDK 64-Bit Server VM (build 17-ea+22-1964, mixed mode, sharing)
   
   Please read the title and description of the pull request. The minimum version required is build 25, yours is 22.


-- 
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 pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861496169


   In the next Panama iteration, there will also be a ready-to use copy method, which has same shape as methods added for bulk copy: https://github.com/openjdk/panama-foreign/pull/555


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860649214


   I will now download the JFR files created by the tool and analyze them locally.


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

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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861046986


   I did more testig and was able to make MMapDirectory work the same speed. This patch fixing the perf issue shows the problems: https://github.com/uschindler/lucene/commit/b057213cc6548ffa29b8e2810f39eb84c50a3bc0
   
   The main issue can be seen in the copying still used by Lucene between mmaped segment and heap:
   ```java
     private static void copySegmentToHeap(MemorySegment src, long srcOffset, byte[] target, int targetOffset, int len) {
       Objects.checkFromIndexSize(srcOffset, len, src.byteSize());
       theUnsafe.copyMemory(null, src.address().toRawLongValue() + srcOffset, 
           target, Unsafe.ARRAY_BYTE_BASE_OFFSET + targetOffset, len);
       //MemorySegment.ofArray(target).asSlice(targetOffset, len).copyFrom(src.asSlice(srcOffset, len));
     }
   ```
   
   If you do the memory copy using Unsafe as this static method shows, the performance is identical:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
                   PKLookup      186.78      (2.2%)      180.11      (1.4%)   -3.6% (  -7% -    0%) 0.000
      BrowseMonthSSDVFacets        4.14      (6.0%)        4.05      (6.3%)   -2.3% ( -13% -   10%) 0.238
   BrowseDayOfYearSSDVFacets        4.03      (4.4%)        3.94      (4.7%)   -2.2% ( -10% -    7%) 0.123
                    Prefix3        5.43     (10.4%)        5.36     (11.7%)   -1.3% ( -21% -   23%) 0.708
                     IntNRQ       41.13      (2.0%)       40.61      (1.8%)   -1.2% (  -4% -    2%) 0.040
            LowSloppyPhrase        5.85      (4.2%)        5.79      (3.8%)   -1.0% (  -8% -    7%) 0.411
           HighSloppyPhrase        9.74      (4.1%)        9.64      (3.9%)   -1.0% (  -8% -    7%) 0.416
                 HighPhrase      226.76      (2.7%)      225.13      (2.9%)   -0.7% (  -6% -    5%) 0.417
                  MedPhrase      204.52      (2.7%)      203.77      (2.5%)   -0.4% (  -5% -    4%) 0.656
                     Fuzzy1       84.24      (4.5%)       84.20      (5.9%)   -0.0% (  -9% -   10%) 0.979
                  LowPhrase       23.33      (2.6%)       23.33      (2.7%)   -0.0% (  -5% -    5%) 0.986
                    Respell       34.95      (1.5%)       34.97      (1.6%)    0.1% (  -3% -    3%) 0.898
                   HighTerm     1128.45      (4.8%)     1129.85      (4.7%)    0.1% (  -8% -   10%) 0.934
          HighTermMonthSort       86.97     (17.0%)       87.24     (14.5%)    0.3% ( -26% -   38%) 0.951
               OrHighNotLow      913.63      (4.0%)      919.28      (3.5%)    0.6% (  -6% -    8%) 0.603
               OrNotHighLow      630.14      (2.8%)      635.01      (3.0%)    0.8% (  -4% -    6%) 0.401
                AndHighHigh       42.87      (4.2%)       43.22      (4.4%)    0.8% (  -7% -    9%) 0.550
                    MedTerm     1504.76      (4.3%)     1519.72      (3.8%)    1.0% (  -6% -    9%) 0.440
            MedSloppyPhrase       38.70      (3.2%)       39.11      (2.6%)    1.1% (  -4% -    7%) 0.251
                 AndHighLow      464.18      (4.4%)      469.92      (4.6%)    1.2% (  -7% -   10%) 0.388
               OrNotHighMed      762.97      (2.8%)      773.13      (3.2%)    1.3% (  -4% -    7%) 0.162
                  OrHighLow      290.51      (4.7%)      294.49      (4.3%)    1.4% (  -7% -   10%) 0.339
                     Fuzzy2       51.19      (8.4%)       51.90      (7.2%)    1.4% ( -13% -   18%) 0.576
                    LowTerm     1892.07      (2.9%)     1919.02      (4.0%)    1.4% (  -5% -    8%) 0.199
                 AndHighMed       43.09      (5.0%)       43.76      (4.7%)    1.6% (  -7% -   11%) 0.311
               OrHighNotMed      708.72      (3.0%)      719.93      (2.7%)    1.6% (  -4% -    7%) 0.084
              OrHighNotHigh      791.29      (4.2%)      803.91      (3.7%)    1.6% (  -6% -    9%) 0.204
                  OrHighMed       37.87      (4.8%)       38.52      (3.2%)    1.7% (  -5% -   10%) 0.179
              OrNotHighHigh      755.35      (3.4%)      769.72      (3.6%)    1.9% (  -4% -    9%) 0.084
      HighTermDayOfYearSort       66.49     (17.2%)       67.81     (15.3%)    2.0% ( -25% -   41%) 0.700
       HighIntervalsOrdered        5.10      (4.0%)        5.20      (3.9%)    2.0% (  -5% -   10%) 0.108
       HighTermTitleBDVSort       71.78     (17.6%)       73.23     (16.8%)    2.0% ( -27% -   44%) 0.712
                 OrHighHigh        9.62      (6.7%)        9.83      (3.2%)    2.2% (  -7% -   13%) 0.181
                   Wildcard       13.65      (9.0%)       14.11     (10.0%)    3.4% ( -14% -   24%) 0.264
                 TermDTSort       61.68     (15.7%)       64.16      (9.1%)    4.0% ( -18% -   34%) 0.324
               HighSpanNear        8.32      (2.6%)        8.72      (3.1%)    4.7% (   0% -   10%) 0.000
                LowSpanNear       10.98      (2.1%)       11.54      (2.2%)    5.1% (   0% -    9%) 0.000
                MedSpanNear        8.34      (2.0%)        8.77      (2.2%)    5.2% (   0% -    9%) 0.000
      BrowseMonthTaxoFacets        1.02      (3.5%)        1.13     (13.6%)   10.1% (  -6% -   28%) 0.001
   BrowseDayOfYearTaxoFacets        1.00      (4.6%)        1.11     (14.4%)   10.5% (  -8% -   30%) 0.002
       BrowseDateTaxoFacets        1.00      (4.6%)        1.11     (14.4%)   10.7% (  -8% -   31%) 0.002
    
   CPU merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_
 version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/bench
 mark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 4.04 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1544656 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   15.33%        236729        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.95%         153617        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   7.58%         117056        org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   5.42%         83667         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   4.47%         68989         jdk.internal.misc.ScopedMemoryAccess#getShortUnalignedInternal()
   4.43%         68353         jdk.internal.foreign.Utils#filterSegment()
   2.49%         38393         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   1.98%         30594         jdk.internal.foreign.AbstractMemorySegmentImpl#checkBoundsSmall()
   1.91%         29460         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.82%         28104         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.73%         26738         jdk.internal.foreign.AbstractMemorySegmentImpl#isSet()
   1.72%         26516         sun.misc.Unsafe#copyMemory()
   1.63%         25197         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#nextDoc()
   1.55%         23880         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
   1.42%         21908         jdk.internal.util.Preconditions#checkFromIndexSize()
   1.31%         20268         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   1.10%         16954         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   1.00%         15398         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.94%         14473         java.lang.invoke.VarHandleGuards#guard_LJ_I()
   0.90%         13916         jdk.internal.foreign.SharedScope#checkValidState()
   0.83%         12820         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.78%         12064         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.73%         11224         org.apache.lucene.index.SingletonSortedSetDocValues#nextOrd()
   0.72%         11055         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.71%         10950         org.apache.lucene.store.DataInput#readVInt()
   0.69%         10643         org.apache.lucene.store.MemorySegmentIndexInput$SingleSegmentImpl#seek()
   0.69%         10612         org.apache.lucene.queries.intervals.OrderedIntervalsSource$OrderedIntervalIterator#nextInterval()
   0.61%         9380          jdk.internal.foreign.AbstractMemorySegmentImpl#scope()
   0.57%         8825          org.apache.lucene.util.BitSet#or()
   0.57%         8747          java.util.Objects#checkIndex()
    
    
   CPU merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-ba
 seline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 4.24 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1712367 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   12.99%        222468        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.56%         163732        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   5.78%         99057         java.nio.ByteBuffer#getArray()
   5.05%         86545         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   4.12%         70481         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   3.96%         67813         java.nio.Buffer#scope()
   3.87%         66312         jdk.internal.misc.Unsafe#convEndian()
   3.81%         65189         org.apache.lucene.store.ByteBufferGuard#ensureValid()
   3.48%         59539         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   2.30%         39343         org.apache.lucene.store.ByteBufferGuard#getBytes()
   2.14%         36573         java.nio.ByteBuffer#get()
   1.88%         32278         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.67%         28624         java.nio.Buffer#checkIndex()
   1.53%         26222         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#nextDoc()
   1.22%         20948         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   1.19%         20403         org.apache.lucene.store.ByteBufferGuard#getShort()
   1.03%         17553         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.01%         17218         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   0.94%         16167         java.nio.Buffer#position()
   0.89%         15218         org.apache.lucene.store.ByteBufferIndexInput#readBytes()
   0.88%         15123         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.76%         12938         org.apache.lucene.store.ByteBufferIndexInput$SingleBufferImpl#seek()
   0.75%         12878         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$19#ordValue()
   0.75%         12813         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.73%         12537         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.69%         11839         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.69%         11738         org.apache.lucene.store.DataInput#readVInt()
   0.65%         11070         org.apache.lucene.queries.intervals.OrderedIntervalsSource$OrderedIntervalIterator#nextInterval()
   0.62%         10575         jdk.internal.util.Preconditions#checkFromIndexSize()
   0.59%         10022         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
    
    
   HEAP merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified
 _version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/benc
 hmark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 2.67 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 68380 events (total: 25828M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   18.44%        4761M         org.apache.lucene.util.FixedBitSet#<init>()
   7.55%         1948M         java.util.AbstractList#iterator()
   5.46%         1409M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   5.21%         1345M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.55%         1174M         org.apache.lucene.util.BytesRef#<init>()
   4.14%         1068M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   3.95%         1020M         org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   3.75%         967M          org.apache.lucene.util.ArrayUtil#growExact()
   3.13%         808M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   2.90%         749M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.07%         534M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   1.90%         492M          java.util.ArrayList#grow()
   1.78%         460M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.43%         369M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.32%         340M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.31%         337M          java.util.AbstractList#listIterator()
   1.27%         328M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.23%         317M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.13%         290M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   1.11%         287M          java.util.ArrayList#iterator()
   1.09%         282M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.07%         276M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.06%         274M          org.apache.lucene.store.MemorySegmentIndexInput#buildSlice()
   1.01%         259M          java.util.Arrays#asList()
   0.97%         250M          java.util.Arrays#copyOf()
   0.96%         248M          org.apache.lucene.util.PriorityQueue#<init>()
   0.91%         233M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.87%         223M          org.apache.lucene.queryparser.classic.Token#newToken()
   0.86%         222M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.84%         216M          jdk.internal.foreign.MappedMemorySegmentImpl#dup()
    
    
   HEAP merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-b
 aseline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 2.58 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 69795 events (total: 26355M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   17.98%        4739M         org.apache.lucene.util.FixedBitSet#<init>()
   7.47%         1968M         java.util.AbstractList#iterator()
   5.16%         1359M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   4.97%         1309M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.44%         1169M         org.apache.lucene.util.BytesRef#<init>()
   4.08%         1074M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   3.91%         1030M         org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   3.38%         892M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   3.36%         885M          org.apache.lucene.util.ArrayUtil#growExact()
   2.81%         741M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.08%         548M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   2.00%         528M          java.util.ArrayList#grow()
   1.82%         479M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.58%         415M          java.nio.DirectByteBufferR#duplicate()
   1.43%         376M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.41%         370M          java.util.AbstractList#listIterator()
   1.33%         350M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.26%         330M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.17%         307M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.16%         306M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.14%         299M          java.util.ArrayList#iterator()
   1.12%         295M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.00%         263M          java.nio.DirectByteBufferR#slice()
   0.98%         257M          java.util.Arrays#asList()
   0.97%         254M          org.apache.lucene.util.PriorityQueue#<init>()
   0.91%         239M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   0.88%         231M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.73%         192M          java.nio.DirectByteBufferR#asLongBuffer()
   0.70%         184M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.68%         179M          org.apache.lucene.store.ByteBufferIndexInput#newCloneInstance()
   ```
   
   If you comment out the unsafe code and use the "official MemorySegmen API", the whole thing gets crazy:
   - runtime of a bench run on my machine oes up from 57 seconds to 75 seconds, while otheriwse (Unsafe) staying identical to baseline
   - in addition it produces a lot of garbage, heap dump contains many `HeapMemorySegmentImpl$OfByte` classes (that are the wrappers around `byte[]` when viewed as MemorySegment. Every wrapping produces a new instance which is not catched by escape analysis. This slows down! The heap dump has 50% of heap filled with those objects according to JFR.
   
   I will report this problem to project Panama! @mcimadamore 


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

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



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


[GitHub] [lucene] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866078727


   > So the benchmarking will stay noisy unless we use JMH.
   
   Maybe we should switch `luceneutil` to JMH?  Or, poach ideas from it?  How is it reducing JVM/hotspot noise without passing unrealistic JVM flags :)


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

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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866087762


   Hi @mikemccand, 
   
   > > I have no idea why the facetting stuff at the beginning of the bench output is so badly behaving with MMapDirectory#v2 on top of project panama, I'll ignore this for now. Maybe @mikemccand has an idea! The rest looks perfectly fine to me from the performance (19 runs).
   > 
   > Yeah that is strange. The impact is sizable. These tasks effectively run a `MatchAllDocsQuery`, and for each hit (doc) decodes the packed delta-coded int ordinals from a `BinaryDocValues` field, incrementing counters in a big `int[]`. To the `MMapDirectory` it should look like it's doing a big sequential scan of that `BinaryDocValues` field for each segment. Surely this should be easy for `mmap` ;) Maybe we should run profiler ... maybe something silly is happening, like we are randomly seeking on every hit unnecessarily ...
   
   This is already understood. The problem is the combination of bad JVM defaults and a problem in bulk copy from off-heap to heap! What you see is garbage collector driving crazy (see my flight control heap dumps). The disabled tiered compilation made hotspot optimizations and escape analysis only kick in after 6 seconds benchmark runtime (of total 60s on my machine). In this time it produced 3 java objects per copy operation (`readBytes(), readLongs(), readFloats()`). With tiered compilation this went down, but still 1 second producing tons of garbage.
   
   With JDK 18 this will be solved in https://github.com/openjdk/panama-foreign/pull/555, where we get a new class `MemoryCopy` that allows to copy data off-heap to on-heap without allocating useless wrapper classes. In this pull request this line is producing 3 temporary objects: https://github.com/apache/lucene/blob/bacfa11940ecf505299ec8a2d43a94d64a106182/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java#L167
   
   ```java
   MemorySegment.ofArray(b).asSlice(offset, len).copyFrom(curSegment.asSlice(curPosition, len));
   ```
   
   From JDK 18 on, it will look like this:
   ```
   MemoryCopy.copyToArray(curSegment, curPosition, b, offset, len);
   ```
   
   This was negotiated today 👍 
   
   In general, we should avoid to copy bytes/longs/floats from mmap to on-heap. With the new vector API coming soon, there's no need for that. Just tell vector API to calculate something (e.g. PFOR or cosine) and give it a MemorySegment slice :-) So we should really really avoid all those stupid copy operations in bulks!!!
   
   To read the whole story check here: https://github.com/openjdk/panama-foreign/pull/555#issuecomment-865672909


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

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



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


[GitHub] [lucene] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866119211


   > The problem with luceneutil is also that it respawns a JVM multiple times.
   
   Hmm, we added multiple JVMs long ago precisely because HotSpot was so unpredictable.  I.e. we had clear examples where HotSpot would paint itself into a corner, compiling e.g. `readVInt` poorly and never re-compiling it, or something, such that no matter how long the benchmark ran, it would never reach as good performance as if you simply restarted the whole JVM and rolled the dice again.  But maybe this situation has been improved and these were somehow early HotSpot bugs/issues and we could really remove multiple JVMs without harming how accurately we can extract the mean/variance performance of all our benchmark tasks?


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

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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866087762


   Hi @mikemccand, 
   
   > > I have no idea why the facetting stuff at the beginning of the bench output is so badly behaving with MMapDirectory#v2 on top of project panama, I'll ignore this for now. Maybe @mikemccand has an idea! The rest looks perfectly fine to me from the performance (19 runs).
   > 
   > Yeah that is strange. The impact is sizable. These tasks effectively run a `MatchAllDocsQuery`, and for each hit (doc) decodes the packed delta-coded int ordinals from a `BinaryDocValues` field, incrementing counters in a big `int[]`. To the `MMapDirectory` it should look like it's doing a big sequential scan of that `BinaryDocValues` field for each segment. Surely this should be easy for `mmap` ;) Maybe we should run profiler ... maybe something silly is happening, like we are randomly seeking on every hit unnecessarily ...
   
   This is already understood. The problem is the combination of bad JVM defaults and a problem in bulk copy from off-heap to heap! What you see is garbage collector driving crazy (see my flight control heap dumps). The disabled tiered compilation made hotspot optimizations and escape analysis only kick in after 6 seconds benchmark runtime (of total 60s on my machine). In this time it produced 3 java objects per copy operation (`readBytes(), readLongs(), readFloats()`). With tiered compilation this went down, but still 1 second producing tons of garbage.
   
   With JDK 18 this will be solved in https://github.com/openjdk/panama-foreign/pull/555, where we get a new class `MemoryCopy` that allows to copy data off-heap to on-heap without allocating useless wrapper classes. In this pull request this line is producing 3 temporary objects: https://github.com/apache/lucene/blob/bacfa11940ecf505299ec8a2d43a94d64a106182/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java#L167
   
   ```java
   MemorySegment.ofArray(b).asSlice(offset, len).copyFrom(curSegment.asSlice(curPosition, len));
   ```
   
   From JDK 18 on, it will look like this:
   ```
   MemoryCopy.copyToArray(curSegment, curPosition, b, offset, len);
   ```
   
   This was negotiated today 👍 
   
   In general, we should avoid to copy bytes/longs/floats from mmap to on-heap. With the new vector API coming soon, there's no need for that. Just tell vector API to calculate something (e.g. PFOR or cosine) and give it a MemorySegment slice :-) So we should really really avoid all those stupid copy operations in bulks!!!
   
   To read the whole story check here: https://github.com/openjdk/panama-foreign/pull/555#issuecomment-865672909 - there are also uptodate benchmarks.


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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       So you query the "able" interface to check for the ability to unwrap and then do 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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-890520954


   > Hi @uschindler,
   > I get the following compilation errors when I build your patch (Gradle build 6.3.8)
   > 
   > > Task :altJvmWarning
   > > NOTE: Alternative java toolchain will be used for compilation and tests:
   > > Project will use Java 17 from: /mnt/c/Users/jatin/home-ubuntu/softwares/jdk-17
   > > Gradle runs with Java 14 from: /usr/lib/jvm/java-14-openjdk-amd64
   > > Task :errorProneSkipped
   > > WARNING: errorprone disabled (skipped on non-nightly runs)
   > > Task :lucene:core:compileJava
   > > warning: using incubating module(s): jdk.incubator.foreign
   > > /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:27: error: cannot find symbol
   > > import jdk.incubator.foreign.ResourceScope;
   > > ^
   > > symbol:   class ResourceScope
   > > location: package jdk.incubator.foreign
   > > /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:51: error: cannot find symbol
   > > final ResourceScope scope;
   > > ^
   > > symbol:   class ResourceScope
   > 
   > Your help will be greatly appriciated.
   > Best Regards
   
   Hi,
   
   the current branch works with the release candidate builds of OpenJDK 17. If you get errors like this, you may possibly use a OpenJDK 17 version before Early Access Build 25.
   
   Ca you post `java -version`and tell us if you maybe compiledyour own version?


-- 
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] jbhateja removed a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
jbhateja removed a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891191240






-- 
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 edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891211684


   Hi,
   
   > > Ca you post `java -version`and tell us if you maybe compiledyour own version?
   
   > Hi @uschindler,
   > java --version
   > openjdk 17-ea 2021-09-14
   > OpenJDK Runtime Environment (build 17-ea+22-1964)
   > OpenJDK 64-Bit Server VM (build 17-ea+22-1964, mixed mode, sharing)
   
   Please read the title and description of the pull request. The minimum version required is build 25, yours is 22.


-- 
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 pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861265227


   I also ran the same with tiered compilation turned on and no `-Xbatch` (java defaults). The results are much better, but still the heap allocations are done.
   
   For long-running benchmarks with modern VMs, switching off tiered and enabling batch mode is a bad idea, as it is different from real world. Sure, results are more predictable, but they are noisy anyways (for different reasons, mainly because we don't use JMH). In addition, as soon as you use MethodHandles or VarHandles (as we do here), in addition to Hotspot's low level optimizations, the Java runtime also does lambda transformations of the MethodHandles and VarHarndles at runtime (it rewrites byte code), which cannot be controlled by the Java command line. All code using java Lambdas, dynamic invokes, MethodHandles, VarHandles (like this one) only work well after long runtime. So the benchmarking will stay noisy unless we use JMH. This got worse recently as Lucene code uses more and more Lambdas.
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
      BrowseMonthTaxoFacets        1.06      (9.0%)        0.92      (7.1%)  -13.3% ( -26% -    3%) 0.000
       BrowseDateTaxoFacets        1.03      (9.5%)        0.91      (8.0%)  -11.8% ( -26% -    6%) 0.000
   BrowseDayOfYearTaxoFacets        1.03      (9.7%)        0.91      (8.0%)  -11.8% ( -26% -    6%) 0.000
                    Respell       54.63      (1.8%)       51.37      (1.2%)   -6.0% (  -8% -   -3%) 0.000
                   PKLookup      206.57      (2.7%)      196.05      (2.3%)   -5.1% (  -9% -    0%) 0.000
                     Fuzzy1       83.20      (1.4%)       78.97      (1.5%)   -5.1% (  -7% -   -2%) 0.000
      HighTermDayOfYearSort       10.39     (17.7%)        9.95     (14.5%)   -4.3% ( -30% -   33%) 0.399
                     Fuzzy2       50.01      (2.7%)       48.48      (2.3%)   -3.1% (  -7% -    1%) 0.000
                  OrHighMed       54.71      (3.6%)       53.16      (4.0%)   -2.8% ( -10% -    4%) 0.019
                 AndHighLow      927.79      (2.8%)      904.39      (5.4%)   -2.5% ( -10% -    5%) 0.063
                  OrHighLow      457.95      (2.2%)      446.92      (4.6%)   -2.4% (  -9% -    4%) 0.036
            LowSloppyPhrase       35.92      (4.1%)       35.13      (3.4%)   -2.2% (  -9% -    5%) 0.066
                    Prefix3       22.95     (12.6%)       22.46     (11.5%)   -2.2% ( -23% -   25%) 0.571
      BrowseMonthSSDVFacets        4.00      (1.4%)        3.91      (3.1%)   -2.1% (  -6% -    2%) 0.006
          HighTermMonthSort       30.76     (21.3%)       30.25     (16.2%)   -1.7% ( -32% -   45%) 0.781
                AndHighHigh       40.44      (5.4%)       39.84      (3.0%)   -1.5% (  -9% -    7%) 0.280
           HighSloppyPhrase       14.55      (4.8%)       14.35      (3.8%)   -1.3% (  -9% -    7%) 0.334
       HighTermTitleBDVSort       12.27     (22.5%)       12.13     (26.8%)   -1.1% ( -41% -   62%) 0.884
            MedSloppyPhrase       22.93      (4.1%)       22.70      (3.3%)   -1.0% (  -8% -    6%) 0.396
                MedSpanNear       14.88      (2.8%)       14.73      (2.6%)   -1.0% (  -6% -    4%) 0.250
                 OrHighHigh       12.71      (4.6%)       12.61      (5.5%)   -0.8% ( -10% -    9%) 0.615
                 AndHighMed       57.17      (5.9%)       56.80      (3.1%)   -0.6% (  -9% -    8%) 0.669
                 TermDTSort       67.79      (9.2%)       67.44     (10.9%)   -0.5% ( -18% -   21%) 0.872
   BrowseDayOfYearSSDVFacets        3.88      (1.7%)        3.86      (3.0%)   -0.5% (  -5% -    4%) 0.544
               HighSpanNear        3.60      (3.7%)        3.60      (3.4%)   -0.1% (  -6% -    7%) 0.936
                  MedPhrase      245.29      (2.5%)      249.61      (2.4%)    1.8% (  -3% -    6%) 0.023
                LowSpanNear      246.46      (4.2%)      252.99      (3.1%)    2.7% (  -4% -   10%) 0.023
       HighIntervalsOrdered       10.07      (6.1%)       10.43      (5.9%)    3.6% (  -7% -   16%) 0.059
               OrNotHighLow     1014.03      (3.2%)     1054.08      (3.3%)    3.9% (  -2% -   10%) 0.000
               OrNotHighMed      832.53      (2.7%)      867.52      (2.9%)    4.2% (  -1% -   10%) 0.000
                   Wildcard       59.59     (23.1%)       62.30     (18.1%)    4.5% ( -29% -   59%) 0.489
                   HighTerm     1310.56      (6.1%)     1370.47      (5.0%)    4.6% (  -6% -   16%) 0.010
                  LowPhrase      177.09      (2.1%)      185.99      (2.8%)    5.0% (   0% -   10%) 0.000
                    MedTerm     1657.72      (6.6%)     1756.16      (4.5%)    5.9% (  -4% -   18%) 0.001
                    LowTerm     1665.17      (3.6%)     1764.34      (3.6%)    6.0% (  -1% -   13%) 0.000
              OrHighNotHigh      928.06      (5.2%)      983.69      (4.7%)    6.0% (  -3% -   16%) 0.000
                 HighPhrase      278.47      (2.5%)      295.20      (3.7%)    6.0% (   0% -   12%) 0.000
              OrNotHighHigh      945.96      (5.0%)     1008.94      (4.1%)    6.7% (  -2% -   16%) 0.000
               OrHighNotLow      963.35      (4.9%)     1035.97      (4.7%)    7.5% (  -1% -   18%) 0.000
               OrHighNotMed     1203.88      (3.9%)     1316.54      (4.3%)    9.4% (   1% -   18%) 0.000
                     IntNRQ      165.56      (7.0%)      181.66     (14.1%)    9.7% ( -10% -   33%) 0.006
   
   CPU merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:+HeapDumpOnOutOfMemoryError -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-14.jfr /home/thetaphi/b
 enchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline
 _vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 3.09 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1507032 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   12.74%        191932        org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   12.50%        188370        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   10.97%        165315        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   6.06%         91320         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   4.75%         71593         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   2.97%         44706         jdk.internal.foreign.AbstractMemorySegmentImpl#checkBounds()
   2.67%         40194         jdk.internal.foreign.AbstractMemorySegmentImpl#checkBoundsSmall()
   2.30%         34656         jdk.internal.misc.ScopedMemoryAccess#getShortUnalignedInternal()
   1.83%         27580         jdk.internal.foreign.AbstractMemorySegmentImpl#isSet()
   1.78%         26834         jdk.internal.foreign.AbstractMemorySegmentImpl#<init>()
   1.67%         25130         org.apache.lucene.store.MemorySegmentIndexInput$SingleSegmentImpl#seek()
   1.50%         22648         org.apache.lucene.store.MemorySegmentIndexInput#readBytes()
   1.28%         19257         jdk.internal.foreign.HeapMemorySegmentImpl$OfByte#fromArray()
   1.17%         17642         org.apache.lucene.search.Weight$DefaultBulkScorer#scoreAll()
   1.15%         17288         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.10%         16524         java.lang.invoke.VarHandleGuards#guard_LJ_I()
   1.06%         16019         jdk.internal.foreign.HeapMemorySegmentImpl#<init>()
   1.03%         15456         org.apache.lucene.store.MemorySegmentIndexInput#ensureOpen()
   0.84%         12628         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.81%         12232         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
   0.77%         11643         jdk.internal.foreign.AbstractMemorySegmentImpl#asSlice()
   0.72%         10884         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#skipPositions()
   0.70%         10522         jdk.internal.foreign.SharedScope#checkValidState()
   0.60%         8991          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   0.59%         8844          org.apache.lucene.store.MemorySegmentIndexInput#readByte()
   0.57%         8581          org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.56%         8393          org.apache.lucene.queries.spans.NearSpansOrdered#stretchToOrder()
   0.56%         8393          jdk.internal.foreign.AbstractMemorySegmentImpl#isSmall()
   0.55%         8343          org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$DenseBinaryDocValues#nextDoc()
   0.53%         7941          java.util.Objects#checkIndex()
   
   
   CPU merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:+HeapDumpOnOutOfMemoryError -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-9.jfr /home/thetaphi/ben
 chmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 2.81 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1429487 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   11.27%        161081        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.67%         138204        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   8.35%         119320        java.nio.ByteBuffer#getArray()
   7.91%         113143        org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   6.74%         96408         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   5.02%         71811         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   3.79%         54165         jdk.internal.misc.Unsafe#convEndian()
   2.35%         33565         java.nio.Buffer#position()
   2.19%         31296         org.apache.lucene.store.ByteBufferIndexInput#readBytes()
   2.00%         28556         jdk.internal.util.Preconditions#checkFromIndexSize()
   1.34%         19210         org.apache.lucene.store.ByteBufferGuard#getShort()
   1.27%         18218         org.apache.lucene.search.Weight$DefaultBulkScorer#scoreAll()
   1.17%         16750         java.nio.Buffer#checkIndex()
   1.14%         16264         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.14%         16239         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$DenseBinaryDocValues#nextDoc()
   0.94%         13392         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.84%         11986         java.nio.DirectByteBuffer#ix()
   0.83%         11804         java.util.ArrayList$Itr#hasNext()
   0.81%         11549         org.apache.lucene.store.ByteBufferGuard#ensureValid()
   0.78%         11147         java.nio.Buffer#scope()
   0.74%         10638         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.73%         10480         org.apache.lucene.store.ByteBufferIndexInput$SingleBufferImpl#seek()
   0.65%         9283          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#skipPositions()
   0.60%         8511          jdk.internal.misc.ScopedMemoryAccess#copyMemory()
   0.59%         8453          org.apache.lucene.search.ConjunctionDISI#doNext()
   0.57%         8125          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   0.51%         7320          org.apache.lucene.queries.spans.NearSpansOrdered#stretchToOrder()
   0.50%         7206          org.apache.lucene.store.ByteBufferGuard#getByte()
   0.48%         6832          org.apache.lucene.queries.spans.TermSpans#nextStartPosition()
   0.47%         6765          org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$14#binaryValue()
   
   
   HEAP merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:+HeapDumpOnOutOfMemoryError -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-14.jfr /home/thetaphi/
 benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/benchmark/util/bench-search-baselin
 e_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 5.80 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 4729188 events (total: 1646537M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   98.51%        1621948M      jdk.internal.foreign.HeapMemorySegmentImpl$OfByte#fromArray()
   0.28%         4680M         org.apache.lucene.util.FixedBitSet#<init>()
   0.09%         1533M         java.util.AbstractList#iterator()
   0.08%         1371M         jdk.internal.foreign.MappedMemorySegmentImpl#dup()
   0.08%         1308M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   0.06%         932M          org.apache.lucene.util.ArrayUtil#growExact()
   0.04%         667M          org.apache.lucene.util.BytesRef#<init>()
   0.04%         659M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   0.04%         622M          jdk.internal.foreign.HeapMemorySegmentImpl$OfLong#dup()
   0.04%         597M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   0.03%         568M          org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   0.03%         524M          org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   0.03%         470M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   0.03%         467M          org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   0.03%         438M          jdk.internal.foreign.HeapMemorySegmentImpl$OfByte#dup()
   0.02%         411M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   0.02%         397M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   0.02%         388M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   0.02%         381M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   0.02%         367M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   0.02%         307M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   0.02%         282M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   0.02%         282M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   0.02%         274M          org.apache.lucene.store.MemorySegmentIndexInput#buildSlice()
   0.02%         252M          java.util.AbstractList#listIterator()
   0.02%         250M          java.util.ArrayList#iterator()
   0.01%         240M          java.util.ArrayList#grow()
   0.01%         216M          java.util.Arrays#copyOfRange()
   0.01%         206M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.01%         199M          java.util.Arrays#asList()
   
   
   HEAP merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:+HeapDumpOnOutOfMemoryError -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-9.jfr /home/thetaphi/be
 nchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 1.32 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 62873 events (total: 23297M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   20.54%        4785M         org.apache.lucene.util.FixedBitSet#<init>()
   6.85%         1596M         java.util.AbstractList#iterator()
   5.78%         1346M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.29%         999M          org.apache.lucene.util.ArrayUtil#growExact()
   3.07%         714M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   2.84%         660M          org.apache.lucene.util.BytesRef#<init>()
   2.78%         648M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.51%         585M          org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   2.30%         536M          org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   2.03%         473M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   1.98%         462M          org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   1.92%         448M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.91%         444M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.89%         441M          java.nio.DirectByteBufferR#duplicate()
   1.72%         399M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.60%         372M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.53%         355M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.22%         284M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.15%         268M          java.util.ArrayList#iterator()
   1.10%         257M          java.util.ArrayList#grow()
   1.08%         250M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.07%         249M          java.util.AbstractList#listIterator()
   1.03%         239M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   1.00%         234M          java.nio.DirectByteBufferR#slice()
   0.92%         214M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.88%         205M          java.util.Arrays#asList()
   0.86%         201M          java.util.Arrays#copyOf()
   0.81%         189M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.81%         187M          org.apache.lucene.store.ByteBufferIndexInput#newCloneInstance()
   0.77%         179M          java.nio.DirectByteBufferR#asLongBuffer()
   ```


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

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



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


[GitHub] [lucene] msokolov commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       yeah, sorry, my comment was mere spelling nit-pick. I only proposed Wrapped (or Wrapper) to avoid negative naming, like booleans that are `notTrue`, but this is different, I don't think anyone will be confused by the "Un"




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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866054411


   Hi @mikemccand, 
   I will open a few more issues in luceneutil. The way how it invokes the JVM is not like it is done in production:
   - `-Xbatch` as default doesn't help for reproducibility and makes the results problematic for production use, as unrealistic.
   - `-XX:-TieredCompilation` as default is only useful to run tests (so we don't spare too much time for recompile), but as it is default in modern JDKS since Java 8, it should really be swritched on for benchmarks. While testing this, disabling tiered compilation made results 20% worse because it prevented escape analysis from kicking in early enough.
   
   So in general the benchmark should mimic real-live. You are repating the runs several times anyways, so changing the optimizations is not a good idea, because your settings do not make fluctuation better, they slow down!


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

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



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


[GitHub] [lucene] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-869081788


   > That's just an idea.
   
   I like this idea!  We could maybe identify "known bad hotspot compilation patterns" and retry the benchmark.
   
   But the problem is (as you pointed out earlier), such mis-compilation also happens in "real" production usage, and including those (annoyingly bad) datapoints in nightly benchmarks would help us gauge how often this happens.


-- 
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 edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861046986


   I did more testig and was able to make MMapDirectory work the same speed. This patch fixing the perf issue shows the problems: https://github.com/uschindler/lucene/commit/b057213cc6548ffa29b8e2810f39eb84c50a3bc0
   
   The main issue can be seen in the copying, still used by Lucene between mmaped segment and heap. Here is the modified code by the patch using `sun.misc.Unsafe` to copy memory instead of the code commented out:
   ```java
     private static void copySegmentToHeap(MemorySegment src, long srcOffset, byte[] target, int targetOffset, int len) {
       Objects.checkFromIndexSize(srcOffset, len, src.byteSize());
       theUnsafe.copyMemory(null, src.address().toRawLongValue() + srcOffset, 
           target, Unsafe.ARRAY_BYTE_BASE_OFFSET + targetOffset, len);
       //MemorySegment.ofArray(target).asSlice(targetOffset, len).copyFrom(src.asSlice(srcOffset, len));
     }
   ```
   
   If you do the memory copy using Unsafe as this static method shows, the performance is identical:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
                   PKLookup      186.78      (2.2%)      180.11      (1.4%)   -3.6% (  -7% -    0%) 0.000
      BrowseMonthSSDVFacets        4.14      (6.0%)        4.05      (6.3%)   -2.3% ( -13% -   10%) 0.238
   BrowseDayOfYearSSDVFacets        4.03      (4.4%)        3.94      (4.7%)   -2.2% ( -10% -    7%) 0.123
                    Prefix3        5.43     (10.4%)        5.36     (11.7%)   -1.3% ( -21% -   23%) 0.708
                     IntNRQ       41.13      (2.0%)       40.61      (1.8%)   -1.2% (  -4% -    2%) 0.040
            LowSloppyPhrase        5.85      (4.2%)        5.79      (3.8%)   -1.0% (  -8% -    7%) 0.411
           HighSloppyPhrase        9.74      (4.1%)        9.64      (3.9%)   -1.0% (  -8% -    7%) 0.416
                 HighPhrase      226.76      (2.7%)      225.13      (2.9%)   -0.7% (  -6% -    5%) 0.417
                  MedPhrase      204.52      (2.7%)      203.77      (2.5%)   -0.4% (  -5% -    4%) 0.656
                     Fuzzy1       84.24      (4.5%)       84.20      (5.9%)   -0.0% (  -9% -   10%) 0.979
                  LowPhrase       23.33      (2.6%)       23.33      (2.7%)   -0.0% (  -5% -    5%) 0.986
                    Respell       34.95      (1.5%)       34.97      (1.6%)    0.1% (  -3% -    3%) 0.898
                   HighTerm     1128.45      (4.8%)     1129.85      (4.7%)    0.1% (  -8% -   10%) 0.934
          HighTermMonthSort       86.97     (17.0%)       87.24     (14.5%)    0.3% ( -26% -   38%) 0.951
               OrHighNotLow      913.63      (4.0%)      919.28      (3.5%)    0.6% (  -6% -    8%) 0.603
               OrNotHighLow      630.14      (2.8%)      635.01      (3.0%)    0.8% (  -4% -    6%) 0.401
                AndHighHigh       42.87      (4.2%)       43.22      (4.4%)    0.8% (  -7% -    9%) 0.550
                    MedTerm     1504.76      (4.3%)     1519.72      (3.8%)    1.0% (  -6% -    9%) 0.440
            MedSloppyPhrase       38.70      (3.2%)       39.11      (2.6%)    1.1% (  -4% -    7%) 0.251
                 AndHighLow      464.18      (4.4%)      469.92      (4.6%)    1.2% (  -7% -   10%) 0.388
               OrNotHighMed      762.97      (2.8%)      773.13      (3.2%)    1.3% (  -4% -    7%) 0.162
                  OrHighLow      290.51      (4.7%)      294.49      (4.3%)    1.4% (  -7% -   10%) 0.339
                     Fuzzy2       51.19      (8.4%)       51.90      (7.2%)    1.4% ( -13% -   18%) 0.576
                    LowTerm     1892.07      (2.9%)     1919.02      (4.0%)    1.4% (  -5% -    8%) 0.199
                 AndHighMed       43.09      (5.0%)       43.76      (4.7%)    1.6% (  -7% -   11%) 0.311
               OrHighNotMed      708.72      (3.0%)      719.93      (2.7%)    1.6% (  -4% -    7%) 0.084
              OrHighNotHigh      791.29      (4.2%)      803.91      (3.7%)    1.6% (  -6% -    9%) 0.204
                  OrHighMed       37.87      (4.8%)       38.52      (3.2%)    1.7% (  -5% -   10%) 0.179
              OrNotHighHigh      755.35      (3.4%)      769.72      (3.6%)    1.9% (  -4% -    9%) 0.084
      HighTermDayOfYearSort       66.49     (17.2%)       67.81     (15.3%)    2.0% ( -25% -   41%) 0.700
       HighIntervalsOrdered        5.10      (4.0%)        5.20      (3.9%)    2.0% (  -5% -   10%) 0.108
       HighTermTitleBDVSort       71.78     (17.6%)       73.23     (16.8%)    2.0% ( -27% -   44%) 0.712
                 OrHighHigh        9.62      (6.7%)        9.83      (3.2%)    2.2% (  -7% -   13%) 0.181
                   Wildcard       13.65      (9.0%)       14.11     (10.0%)    3.4% ( -14% -   24%) 0.264
                 TermDTSort       61.68     (15.7%)       64.16      (9.1%)    4.0% ( -18% -   34%) 0.324
               HighSpanNear        8.32      (2.6%)        8.72      (3.1%)    4.7% (   0% -   10%) 0.000
                LowSpanNear       10.98      (2.1%)       11.54      (2.2%)    5.1% (   0% -    9%) 0.000
                MedSpanNear        8.34      (2.0%)        8.77      (2.2%)    5.2% (   0% -    9%) 0.000
      BrowseMonthTaxoFacets        1.02      (3.5%)        1.13     (13.6%)   10.1% (  -6% -   28%) 0.001
   BrowseDayOfYearTaxoFacets        1.00      (4.6%)        1.11     (14.4%)   10.5% (  -8% -   30%) 0.002
       BrowseDateTaxoFacets        1.00      (4.6%)        1.11     (14.4%)   10.7% (  -8% -   31%) 0.002
    
   CPU merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_
 version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/bench
 mark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 4.04 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1544656 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   15.33%        236729        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.95%         153617        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   7.58%         117056        org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   5.42%         83667         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   4.47%         68989         jdk.internal.misc.ScopedMemoryAccess#getShortUnalignedInternal()
   4.43%         68353         jdk.internal.foreign.Utils#filterSegment()
   2.49%         38393         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   1.98%         30594         jdk.internal.foreign.AbstractMemorySegmentImpl#checkBoundsSmall()
   1.91%         29460         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.82%         28104         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.73%         26738         jdk.internal.foreign.AbstractMemorySegmentImpl#isSet()
   1.72%         26516         sun.misc.Unsafe#copyMemory()
   1.63%         25197         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#nextDoc()
   1.55%         23880         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
   1.42%         21908         jdk.internal.util.Preconditions#checkFromIndexSize()
   1.31%         20268         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   1.10%         16954         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   1.00%         15398         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.94%         14473         java.lang.invoke.VarHandleGuards#guard_LJ_I()
   0.90%         13916         jdk.internal.foreign.SharedScope#checkValidState()
   0.83%         12820         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.78%         12064         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.73%         11224         org.apache.lucene.index.SingletonSortedSetDocValues#nextOrd()
   0.72%         11055         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.71%         10950         org.apache.lucene.store.DataInput#readVInt()
   0.69%         10643         org.apache.lucene.store.MemorySegmentIndexInput$SingleSegmentImpl#seek()
   0.69%         10612         org.apache.lucene.queries.intervals.OrderedIntervalsSource$OrderedIntervalIterator#nextInterval()
   0.61%         9380          jdk.internal.foreign.AbstractMemorySegmentImpl#scope()
   0.57%         8825          org.apache.lucene.util.BitSet#or()
   0.57%         8747          java.util.Objects#checkIndex()
    
    
   CPU merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-ba
 seline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 4.24 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1712367 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   12.99%        222468        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.56%         163732        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   5.78%         99057         java.nio.ByteBuffer#getArray()
   5.05%         86545         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   4.12%         70481         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   3.96%         67813         java.nio.Buffer#scope()
   3.87%         66312         jdk.internal.misc.Unsafe#convEndian()
   3.81%         65189         org.apache.lucene.store.ByteBufferGuard#ensureValid()
   3.48%         59539         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   2.30%         39343         org.apache.lucene.store.ByteBufferGuard#getBytes()
   2.14%         36573         java.nio.ByteBuffer#get()
   1.88%         32278         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.67%         28624         java.nio.Buffer#checkIndex()
   1.53%         26222         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#nextDoc()
   1.22%         20948         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   1.19%         20403         org.apache.lucene.store.ByteBufferGuard#getShort()
   1.03%         17553         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.01%         17218         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   0.94%         16167         java.nio.Buffer#position()
   0.89%         15218         org.apache.lucene.store.ByteBufferIndexInput#readBytes()
   0.88%         15123         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.76%         12938         org.apache.lucene.store.ByteBufferIndexInput$SingleBufferImpl#seek()
   0.75%         12878         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$19#ordValue()
   0.75%         12813         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.73%         12537         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.69%         11839         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.69%         11738         org.apache.lucene.store.DataInput#readVInt()
   0.65%         11070         org.apache.lucene.queries.intervals.OrderedIntervalsSource$OrderedIntervalIterator#nextInterval()
   0.62%         10575         jdk.internal.util.Preconditions#checkFromIndexSize()
   0.59%         10022         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
    
    
   HEAP merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified
 _version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/benc
 hmark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 2.67 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 68380 events (total: 25828M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   18.44%        4761M         org.apache.lucene.util.FixedBitSet#<init>()
   7.55%         1948M         java.util.AbstractList#iterator()
   5.46%         1409M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   5.21%         1345M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.55%         1174M         org.apache.lucene.util.BytesRef#<init>()
   4.14%         1068M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   3.95%         1020M         org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   3.75%         967M          org.apache.lucene.util.ArrayUtil#growExact()
   3.13%         808M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   2.90%         749M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.07%         534M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   1.90%         492M          java.util.ArrayList#grow()
   1.78%         460M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.43%         369M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.32%         340M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.31%         337M          java.util.AbstractList#listIterator()
   1.27%         328M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.23%         317M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.13%         290M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   1.11%         287M          java.util.ArrayList#iterator()
   1.09%         282M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.07%         276M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.06%         274M          org.apache.lucene.store.MemorySegmentIndexInput#buildSlice()
   1.01%         259M          java.util.Arrays#asList()
   0.97%         250M          java.util.Arrays#copyOf()
   0.96%         248M          org.apache.lucene.util.PriorityQueue#<init>()
   0.91%         233M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.87%         223M          org.apache.lucene.queryparser.classic.Token#newToken()
   0.86%         222M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.84%         216M          jdk.internal.foreign.MappedMemorySegmentImpl#dup()
    
    
   HEAP merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-b
 aseline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 2.58 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 69795 events (total: 26355M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   17.98%        4739M         org.apache.lucene.util.FixedBitSet#<init>()
   7.47%         1968M         java.util.AbstractList#iterator()
   5.16%         1359M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   4.97%         1309M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.44%         1169M         org.apache.lucene.util.BytesRef#<init>()
   4.08%         1074M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   3.91%         1030M         org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   3.38%         892M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   3.36%         885M          org.apache.lucene.util.ArrayUtil#growExact()
   2.81%         741M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.08%         548M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   2.00%         528M          java.util.ArrayList#grow()
   1.82%         479M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.58%         415M          java.nio.DirectByteBufferR#duplicate()
   1.43%         376M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.41%         370M          java.util.AbstractList#listIterator()
   1.33%         350M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.26%         330M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.17%         307M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.16%         306M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.14%         299M          java.util.ArrayList#iterator()
   1.12%         295M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.00%         263M          java.nio.DirectByteBufferR#slice()
   0.98%         257M          java.util.Arrays#asList()
   0.97%         254M          org.apache.lucene.util.PriorityQueue#<init>()
   0.91%         239M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   0.88%         231M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.73%         192M          java.nio.DirectByteBufferR#asLongBuffer()
   0.70%         184M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.68%         179M          org.apache.lucene.store.ByteBufferIndexInput#newCloneInstance()
   ```
   
   If you comment out the unsafe code and use the "official MemorySegmen API", the whole thing gets crazy:
   - runtime of a bench run on my machine oes up from 57 seconds to 75 seconds, while otheriwse (Unsafe) staying identical to baseline
   - in addition it produces a lot of garbage, heap dump contains many `HeapMemorySegmentImpl$OfByte` classes (that are the wrappers around `byte[]` when viewed as MemorySegment. Every wrapping produces a new instance which is not catched by escape analysis. This slows down! The heap dump has 50% of heap filled with those objects according to JFR.
   
   I will report this problem to project Panama! @mcimadamore 


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

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



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


[GitHub] [lucene] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866046843


   > @mikemccand It would be better to have the tool respect `RUNTIME_JAVA_HOME` like gradlew does.
   
   +1, that sounds like a good idea!  Maybe open an issue in `luceneutil` directly?


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

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



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


[GitHub] [lucene] uschindler removed a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler removed a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860731308


   I used the wrong benchmark task (not wikimediumall but wikimedium10k, copypastershitnwaste). New benchmarks coming soon.


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866087762


   Hi @mikemccand, 
   
   > > I have no idea why the facetting stuff at the beginning of the bench output is so badly behaving with MMapDirectory#v2 on top of project panama, I'll ignore this for now. Maybe @mikemccand has an idea! The rest looks perfectly fine to me from the performance (19 runs).
   > 
   > Yeah that is strange. The impact is sizable. These tasks effectively run a `MatchAllDocsQuery`, and for each hit (doc) decodes the packed delta-coded int ordinals from a `BinaryDocValues` field, incrementing counters in a big `int[]`. To the `MMapDirectory` it should look like it's doing a big sequential scan of that `BinaryDocValues` field for each segment. Surely this should be easy for `mmap` ;) Maybe we should run profiler ... maybe something silly is happening, like we are randomly seeking on every hit unnecessarily ...
   
   This is already understood. The problem is the combination of bad JVM defaults and a problem in bulk copy from off-heap to heap! What you see is garbage collector driving crazy (see my flight control heap dumps). The disabled tiered compilation made hotspot optimizations and escape analysis only kick in after 6 seconds benchmark runtime (of total 60s on my machine). In this time it produced 3 java objects per copy operation (`readBytes(), readLongs(), readFloats()`). With tiered compilation this went down, but still 1 second producing tons of garbage.
   
   With JDK 18 this will be solved in https://github.com/openjdk/panama-foreign/pull/555, where we get a new class `MemoryCopy` that allows to copy data off-heap to on-heap without allocating useless wrapper classes. In this pull request this line is producing 3 temporary objects: https://github.com/apache/lucene/blob/bacfa11940ecf505299ec8a2d43a94d64a106182/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java#L167
   
   ```java
   MemorySegment.ofArray(b).asSlice(offset, len).copyFrom(curSegment.asSlice(curPosition, len));
   ```
   
   From JDK 18 on, it will look like this:
   ```
   MemoryCopy.copyToArray(curSegment, curPosition, b, offset, len);
   ```
   
   This was negotiated today 👍 
   
   To read the whole story check here: https://github.com/openjdk/panama-foreign/pull/555#issuecomment-865672909


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866980038


   > > But I think it would be improved by providing some more diagnostics (LogCompilation or whatever, maybe JIT stats in the JFR output). Let it be a "canary" to find little ways to improve.
   > 
   > +1. I wonder if we could tap into those in real-time and get a sense of when the JVM really is roughly "warmed up", instead of the static "discard first N samples for each task" that we do now. Or maybe to detect mis-compilation of `readVInt`!
   
   Unfortunately, you can't get the compilation events from inside the JVM, but with the help of the outer python process it might be possible:
   
   The inner java process just benchmarks every round/query and does not throw away anything. After each round it prints the information in "machine readable form" to stdout. In addition we turn on `-XX:+PrintCompilation` on the JVM command line.
   
   The outer python process just reads process output and reacts to events:
   - if a benchmark query was finished it records the machine readable number
   - if it gets a compilation event on stdout (some regex can catch it), it greps for some "hot method" like "readVInt" and once it sees a compilation event (with tiered you jave to look for compilation stage 4, also known as C2), it switches the flag and from now on it can use the numbers recorded
   
   That's just an idea.


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861362120


   I opened https://bugs.openjdk.java.net/browse/JDK-8268743 about the object allocations.


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

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



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


[GitHub] [lucene] jbhateja commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
jbhateja commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891467513


   > Hi,
   > 
   > > > Ca you post `java -version`and tell us if you maybe compiledyour own version?
   > 
   > > Hi @uschindler,
   > > java --version
   > > openjdk 17-ea 2021-09-14
   > > OpenJDK Runtime Environment (build 17-ea+22-1964)
   > > OpenJDK 64-Bit Server VM (build 17-ea+22-1964, mixed mode, sharing)
   > 
   > Please read the title and description of the pull request. The minimum version required is build 25, yours is 22.
   
   Hi @uschindler, I see the same issue with build 17-ea+33-2705. Just curious why is (>= JDK-17-ea-b25) version must for this patch since jdk.foreign.incubator libraries existed earlier.
   Best Regards
   


-- 
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] rmuir commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866272233


   > Hmm, we added multiple JVMs long ago precisely because HotSpot was so unpredictable. I.e. we had clear examples where HotSpot would paint itself into a corner, compiling e.g. readVInt poorly and never re-compiling it, or something, such that no matter how long the benchmark ran, it would never reach as good performance as if you simply restarted the whole JVM and rolled the dice again. But maybe this situation has been improved and these were somehow early HotSpot bugs/issues and we could really remove multiple JVMs without harming how accurately we can extract the mean/variance performance of all our benchmark tasks?
   
   the JMH will do this too. I forget the defaults, but uses multiple jvm iterations and iterations within each jvm and warmup iterations. But it has smarts around the JIT compiler and can dump profiled assembly for its microbenchmarks. I never have noise issues with it.
   
   I think the issue is like "unit test" versus "integration test". 
   
   The current big "integration test" (lucene util) is useful for some things: e.g. something has to tell us there is pollution from too many java abstractions going megamorphic and so on :) But I think it would be improved by providing some more diagnostics (LogCompilation or whatever, maybe JIT stats in the JFR output). Let it be a "canary" to find little ways to improve.
   
   But we have nothing setup to do simple noise-free microbenchmarks over some specific code, e.g. like "unit tests" running different query types. And for those you don't want crazy JFR and logging and stuff as it is so targeted, you can just dump the hot assembly code instead. For now if you want to do this, you are writing one-off stuff yourself.
   


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

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



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


[GitHub] [lucene] jbhateja removed a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
jbhateja removed a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891467513


   > Hi,
   > 
   > > > Ca you post `java -version`and tell us if you maybe compiledyour own version?
   > 
   > > Hi @uschindler,
   > > java --version
   > > openjdk 17-ea 2021-09-14
   > > OpenJDK Runtime Environment (build 17-ea+22-1964)
   > > OpenJDK 64-Bit Server VM (build 17-ea+22-1964, mixed mode, sharing)
   > 
   > Please read the title and description of the pull request. The minimum version required is build 25, yours is 22.
   
   Hi @uschindler, I see the same issue with build 17-ea+33-2705. Just curious why is (>= JDK-17-ea-b25) version must for this patch since jdk.foreign.incubator libraries existed earlier.
   Best Regards
   


-- 
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] jbhateja commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
jbhateja commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-890408686


   Hi @uschindler,
   I get the following compilation errors when I build your patch (Gradle build 6.3.8)
   > Task :altJvmWarning
   NOTE: Alternative java toolchain will be used for compilation and tests:
     Project will use Java 17 from: /mnt/c/Users/jatin/home-ubuntu/softwares/jdk-17
     Gradle runs with Java 14 from: /usr/lib/jvm/java-14-openjdk-amd64
   > Task :errorProneSkipped
   WARNING: errorprone disabled (skipped on non-nightly runs)
   > Task :lucene:core:compileJava
   warning: using incubating module(s): jdk.incubator.foreign
   /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:27: error: cannot find symbol
   import jdk.incubator.foreign.ResourceScope;
                               ^
     symbol:   class ResourceScope
     location: package jdk.incubator.foreign
   /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:51: error: cannot find symbol
     final ResourceScope scope;
           ^
     symbol:   class ResourceScope
    
   Your help will be greatly appriciated. 
   Best Regards 


-- 
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 pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866088334


   > > So the benchmarking will stay noisy unless we use JMH.
   > 
   > Maybe we should switch `luceneutil` to JMH? Or, poach ideas from it? How is it reducing JVM/hotspot noise without passing unrealistic JVM flags :)
   
   I discussed with Robert about this. Yes, we should do this! But still we should do "real-world" benchmarks.


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

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



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


[GitHub] [lucene] msokolov commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {
+      return alreadyClosed();
+    }
+    // TODO: maybe open a JDK issue to have a separate, more
+    // meaningful exception for unmapped segments:
+    if (e.getMessage() != null && e.getMessage().contains("closed")) {
+      return alreadyClosed();
+    }
+    return e;
+  }
+
+  RuntimeException handlePositionalIOOBE(String action, long pos) throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position: " + this);
+    } else {
+      throw new EOFException(action + " past EOF: " + this);
+    }
+  }
+
+  private AlreadyClosedException alreadyClosed() {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      do {
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+      } while (curSegment.byteSize() == 0L);
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    final MemorySegment arraySegment = MemorySegment.ofArray(b);
+    try {
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(arraySegment, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  private void readBytesBoundary(final MemorySegment arraySegment, int offset, int len)
+      throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, curAvail));
+        len -= curAvail;
+        offset += curAvail;
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+        curAvail = curSegment.byteSize();
+      }
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public void readLongs(long[] dst, int offset, int length) throws IOException {
+    if (IS_LITTLE_ENDIAN) {
+      final MemorySegment targetSlice =
+          MemorySegment.ofArray(dst).asSlice((long) offset << 3, (long) length << 3);
+      try {
+        targetSlice.copyFrom(curSegment.asSlice(curPosition, targetSlice.byteSize()));
+        curPosition += targetSlice.byteSize();
+      } catch (
+          @SuppressWarnings("unused")
+          IndexOutOfBoundsException iobe) {
+        super.readLongs(dst, offset, length);

Review comment:
       thanks for the explanations




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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {

Review comment:
       This triggers the NPE: https://github.com/apache/lucene/pull/177/files/6d16a8d0984c73ddb32fced02a180b1a688662c1#diff-1c8534e50633cd8648f2cb3c886a269a6ebf8a57d817885bd962225d83f4e93fR454-R461
   
   On close() the curSegment is nulled and also the `segments[]` array contents.




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

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



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


[GitHub] [lucene] jbhateja commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
jbhateja commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-890407948


   Hi @uschindler ,
   I tried to build your patch but get the following error. Can you kindly suggest what could be the issue.
   
   Error log:-
   
   > Task :altJvmWarning
   NOTE: Alternative java toolchain will be used for compilation and tests:
     Project will use Java 17 from: /mnt/c/Users/jatin/home-ubuntu/softwares/jdk-17
     Gradle runs with Java 14 from: /usr/lib/jvm/java-14-openjdk-amd64
   
   
   > Task :errorProneSkipped
   WARNING: errorprone disabled (skipped on non-nightly runs)
   
   > Task :lucene:core:compileJava
   warning: using incubating module(s): jdk.incubator.foreign
   /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:27: error: cannot find symbol
   import jdk.incubator.foreign.ResourceScope;
                               ^
     symbol:   class ResourceScope
     location: package jdk.incubator.foreign
   /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:51: error: cannot find symbol
     final ResourceScope scope;
           ^
     symbol:   class ResourceScope
   
   Thanks in advance.


-- 
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] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866067673


   > I have no idea why the facetting stuff at the beginning of the bench output is so badly behaving with MMapDirectory#v2 on top of project panama, I'll ignore this for now. Maybe @mikemccand has an idea! The rest looks perfectly fine to me from the performance (19 runs).
   
   Yeah that is strange.  The impact is sizable.  These tasks effectively run a `MatchAllDocsQuery`, and for each hit (doc) decodes the packed delta-coded int ordinals from a `BinaryDocValues` field, incrementing counters in a big `int[]`.  To the `MMapDirectory` it should look like it's doing a big sequential scan of that `BinaryDocValues` field for each segment.  Surely this should be easy for `mmap` ;)  Maybe we should run profiler ... maybe something silly is happening, like we are randomly seeking on every hit unnecessarily ...


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

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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861046986


   I did more testig and was able to make MMapDirectory work the same speed. This patch fixing the perf issue shows the problems: https://github.com/uschindler/lucene/commit/b057213cc6548ffa29b8e2810f39eb84c50a3bc0
   
   The main issue can be seen in the copying, still used by Lucene between mmaped segment and heap. Here is the modified code by the patch using `sun.misc.Unsafe` to copy memory instead of the code commented out:
   ```java
     private static void copySegmentToHeap(MemorySegment src, long srcOffset, byte[] target, int targetOffset, int len) {
       Objects.checkFromIndexSize(srcOffset, len, src.byteSize());
       theUnsafe.copyMemory(null, src.address().toRawLongValue() + srcOffset, 
           target, Unsafe.ARRAY_BYTE_BASE_OFFSET + targetOffset, len);
       //MemorySegment.ofArray(target).asSlice(targetOffset, len).copyFrom(src.asSlice(srcOffset, len));
     }
   ```
   
   If you do the memory copy using Unsafe as this static method shows, the performance is identical:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
                   PKLookup      186.78      (2.2%)      180.11      (1.4%)   -3.6% (  -7% -    0%) 0.000
      BrowseMonthSSDVFacets        4.14      (6.0%)        4.05      (6.3%)   -2.3% ( -13% -   10%) 0.238
   BrowseDayOfYearSSDVFacets        4.03      (4.4%)        3.94      (4.7%)   -2.2% ( -10% -    7%) 0.123
                    Prefix3        5.43     (10.4%)        5.36     (11.7%)   -1.3% ( -21% -   23%) 0.708
                     IntNRQ       41.13      (2.0%)       40.61      (1.8%)   -1.2% (  -4% -    2%) 0.040
            LowSloppyPhrase        5.85      (4.2%)        5.79      (3.8%)   -1.0% (  -8% -    7%) 0.411
           HighSloppyPhrase        9.74      (4.1%)        9.64      (3.9%)   -1.0% (  -8% -    7%) 0.416
                 HighPhrase      226.76      (2.7%)      225.13      (2.9%)   -0.7% (  -6% -    5%) 0.417
                  MedPhrase      204.52      (2.7%)      203.77      (2.5%)   -0.4% (  -5% -    4%) 0.656
                     Fuzzy1       84.24      (4.5%)       84.20      (5.9%)   -0.0% (  -9% -   10%) 0.979
                  LowPhrase       23.33      (2.6%)       23.33      (2.7%)   -0.0% (  -5% -    5%) 0.986
                    Respell       34.95      (1.5%)       34.97      (1.6%)    0.1% (  -3% -    3%) 0.898
                   HighTerm     1128.45      (4.8%)     1129.85      (4.7%)    0.1% (  -8% -   10%) 0.934
          HighTermMonthSort       86.97     (17.0%)       87.24     (14.5%)    0.3% ( -26% -   38%) 0.951
               OrHighNotLow      913.63      (4.0%)      919.28      (3.5%)    0.6% (  -6% -    8%) 0.603
               OrNotHighLow      630.14      (2.8%)      635.01      (3.0%)    0.8% (  -4% -    6%) 0.401
                AndHighHigh       42.87      (4.2%)       43.22      (4.4%)    0.8% (  -7% -    9%) 0.550
                    MedTerm     1504.76      (4.3%)     1519.72      (3.8%)    1.0% (  -6% -    9%) 0.440
            MedSloppyPhrase       38.70      (3.2%)       39.11      (2.6%)    1.1% (  -4% -    7%) 0.251
                 AndHighLow      464.18      (4.4%)      469.92      (4.6%)    1.2% (  -7% -   10%) 0.388
               OrNotHighMed      762.97      (2.8%)      773.13      (3.2%)    1.3% (  -4% -    7%) 0.162
                  OrHighLow      290.51      (4.7%)      294.49      (4.3%)    1.4% (  -7% -   10%) 0.339
                     Fuzzy2       51.19      (8.4%)       51.90      (7.2%)    1.4% ( -13% -   18%) 0.576
                    LowTerm     1892.07      (2.9%)     1919.02      (4.0%)    1.4% (  -5% -    8%) 0.199
                 AndHighMed       43.09      (5.0%)       43.76      (4.7%)    1.6% (  -7% -   11%) 0.311
               OrHighNotMed      708.72      (3.0%)      719.93      (2.7%)    1.6% (  -4% -    7%) 0.084
              OrHighNotHigh      791.29      (4.2%)      803.91      (3.7%)    1.6% (  -6% -    9%) 0.204
                  OrHighMed       37.87      (4.8%)       38.52      (3.2%)    1.7% (  -5% -   10%) 0.179
              OrNotHighHigh      755.35      (3.4%)      769.72      (3.6%)    1.9% (  -4% -    9%) 0.084
      HighTermDayOfYearSort       66.49     (17.2%)       67.81     (15.3%)    2.0% ( -25% -   41%) 0.700
       HighIntervalsOrdered        5.10      (4.0%)        5.20      (3.9%)    2.0% (  -5% -   10%) 0.108
       HighTermTitleBDVSort       71.78     (17.6%)       73.23     (16.8%)    2.0% ( -27% -   44%) 0.712
                 OrHighHigh        9.62      (6.7%)        9.83      (3.2%)    2.2% (  -7% -   13%) 0.181
                   Wildcard       13.65      (9.0%)       14.11     (10.0%)    3.4% ( -14% -   24%) 0.264
                 TermDTSort       61.68     (15.7%)       64.16      (9.1%)    4.0% ( -18% -   34%) 0.324
               HighSpanNear        8.32      (2.6%)        8.72      (3.1%)    4.7% (   0% -   10%) 0.000
                LowSpanNear       10.98      (2.1%)       11.54      (2.2%)    5.1% (   0% -    9%) 0.000
                MedSpanNear        8.34      (2.0%)        8.77      (2.2%)    5.2% (   0% -    9%) 0.000
      BrowseMonthTaxoFacets        1.02      (3.5%)        1.13     (13.6%)   10.1% (  -6% -   28%) 0.001
   BrowseDayOfYearTaxoFacets        1.00      (4.6%)        1.11     (14.4%)   10.5% (  -8% -   30%) 0.002
       BrowseDateTaxoFacets        1.00      (4.6%)        1.11     (14.4%)   10.7% (  -8% -   31%) 0.002
    
   CPU merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_
 version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/bench
 mark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 4.04 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1544656 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   15.33%        236729        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.95%         153617        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   7.58%         117056        org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   5.42%         83667         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   4.47%         68989         jdk.internal.misc.ScopedMemoryAccess#getShortUnalignedInternal()
   4.43%         68353         jdk.internal.foreign.Utils#filterSegment()
   2.49%         38393         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   1.98%         30594         jdk.internal.foreign.AbstractMemorySegmentImpl#checkBoundsSmall()
   1.91%         29460         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.82%         28104         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.73%         26738         jdk.internal.foreign.AbstractMemorySegmentImpl#isSet()
   1.72%         26516         sun.misc.Unsafe#copyMemory()
   1.63%         25197         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#nextDoc()
   1.55%         23880         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
   1.42%         21908         jdk.internal.util.Preconditions#checkFromIndexSize()
   1.31%         20268         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   1.10%         16954         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   1.00%         15398         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.94%         14473         java.lang.invoke.VarHandleGuards#guard_LJ_I()
   0.90%         13916         jdk.internal.foreign.SharedScope#checkValidState()
   0.83%         12820         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.78%         12064         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.73%         11224         org.apache.lucene.index.SingletonSortedSetDocValues#nextOrd()
   0.72%         11055         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.71%         10950         org.apache.lucene.store.DataInput#readVInt()
   0.69%         10643         org.apache.lucene.store.MemorySegmentIndexInput$SingleSegmentImpl#seek()
   0.69%         10612         org.apache.lucene.queries.intervals.OrderedIntervalsSource$OrderedIntervalIterator#nextInterval()
   0.61%         9380          jdk.internal.foreign.AbstractMemorySegmentImpl#scope()
   0.57%         8825          org.apache.lucene.util.BitSet#or()
   0.57%         8747          java.util.Objects#checkIndex()
    
    
   CPU merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-ba
 seline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 4.24 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1712367 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   12.99%        222468        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.56%         163732        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   5.78%         99057         java.nio.ByteBuffer#getArray()
   5.05%         86545         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   4.12%         70481         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   3.96%         67813         java.nio.Buffer#scope()
   3.87%         66312         jdk.internal.misc.Unsafe#convEndian()
   3.81%         65189         org.apache.lucene.store.ByteBufferGuard#ensureValid()
   3.48%         59539         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   2.30%         39343         org.apache.lucene.store.ByteBufferGuard#getBytes()
   2.14%         36573         java.nio.ByteBuffer#get()
   1.88%         32278         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.67%         28624         java.nio.Buffer#checkIndex()
   1.53%         26222         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#nextDoc()
   1.22%         20948         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   1.19%         20403         org.apache.lucene.store.ByteBufferGuard#getShort()
   1.03%         17553         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.01%         17218         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   0.94%         16167         java.nio.Buffer#position()
   0.89%         15218         org.apache.lucene.store.ByteBufferIndexInput#readBytes()
   0.88%         15123         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.76%         12938         org.apache.lucene.store.ByteBufferIndexInput$SingleBufferImpl#seek()
   0.75%         12878         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$19#ordValue()
   0.75%         12813         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.73%         12537         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.69%         11839         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.69%         11738         org.apache.lucene.store.DataInput#readVInt()
   0.65%         11070         org.apache.lucene.queries.intervals.OrderedIntervalsSource$OrderedIntervalIterator#nextInterval()
   0.62%         10575         jdk.internal.util.Preconditions#checkFromIndexSize()
   0.59%         10022         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
    
    
   HEAP merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified
 _version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/benc
 hmark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 2.67 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 68380 events (total: 25828M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   18.44%        4761M         org.apache.lucene.util.FixedBitSet#<init>()
   7.55%         1948M         java.util.AbstractList#iterator()
   5.46%         1409M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   5.21%         1345M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.55%         1174M         org.apache.lucene.util.BytesRef#<init>()
   4.14%         1068M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   3.95%         1020M         org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   3.75%         967M          org.apache.lucene.util.ArrayUtil#growExact()
   3.13%         808M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   2.90%         749M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.07%         534M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   1.90%         492M          java.util.ArrayList#grow()
   1.78%         460M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.43%         369M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.32%         340M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.31%         337M          java.util.AbstractList#listIterator()
   1.27%         328M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.23%         317M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.13%         290M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   1.11%         287M          java.util.ArrayList#iterator()
   1.09%         282M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.07%         276M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.06%         274M          org.apache.lucene.store.MemorySegmentIndexInput#buildSlice()
   1.01%         259M          java.util.Arrays#asList()
   0.97%         250M          java.util.Arrays#copyOf()
   0.96%         248M          org.apache.lucene.util.PriorityQueue#<init>()
   0.91%         233M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.87%         223M          org.apache.lucene.queryparser.classic.Token#newToken()
   0.86%         222M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.84%         216M          jdk.internal.foreign.MappedMemorySegmentImpl#dup()
    
    
   HEAP merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-b
 aseline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 2.58 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 69795 events (total: 26355M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   17.98%        4739M         org.apache.lucene.util.FixedBitSet#<init>()
   7.47%         1968M         java.util.AbstractList#iterator()
   5.16%         1359M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   4.97%         1309M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.44%         1169M         org.apache.lucene.util.BytesRef#<init>()
   4.08%         1074M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   3.91%         1030M         org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   3.38%         892M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   3.36%         885M          org.apache.lucene.util.ArrayUtil#growExact()
   2.81%         741M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.08%         548M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   2.00%         528M          java.util.ArrayList#grow()
   1.82%         479M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.58%         415M          java.nio.DirectByteBufferR#duplicate()
   1.43%         376M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.41%         370M          java.util.AbstractList#listIterator()
   1.33%         350M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.26%         330M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.17%         307M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.16%         306M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.14%         299M          java.util.ArrayList#iterator()
   1.12%         295M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.00%         263M          java.nio.DirectByteBufferR#slice()
   0.98%         257M          java.util.Arrays#asList()
   0.97%         254M          org.apache.lucene.util.PriorityQueue#<init>()
   0.91%         239M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   0.88%         231M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.73%         192M          java.nio.DirectByteBufferR#asLongBuffer()
   0.70%         184M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.68%         179M          org.apache.lucene.store.ByteBufferIndexInput#newCloneInstance()
   ```
   
   If you comment out the unsafe code and use the "official MemorySegment API", the whole thing gets crazy:
   - runtime of one bench run of modified code on my machine goes up from 57 seconds to 75 seconds, while otherwise (Unsafe) staying identical to baseline
   - in addition it produces a lot of garbage, heap dump contains many `HeapMemorySegmentImpl$OfByte` classes (that are the wrappers around `byte[]` when viewed as MemorySegment. Every wrapping produces a new instance which is not catched by escape analysis. This slows down! The heap dump has 50% of heap filled with those objects according to JFR.
   
   I will report this problem to project Panama! @mcimadamore 


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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -231,55 +224,60 @@ public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
     ensureCanRead(name);
     Path path = directory.resolve(name);
-    try (FileChannel c = FileChannel.open(path, StandardOpenOption.READ)) {
-      final String resourceDescription = "MMapIndexInput(path=\"" + path.toString() + "\")";
-      final boolean useUnmap = getUseUnmap();
-      return ByteBufferIndexInput.newInstance(
-          resourceDescription,
-          map(resourceDescription, c, 0, c.size()),
-          c.size(),
-          chunkSizePower,
-          new ByteBufferGuard(resourceDescription, useUnmap ? CLEANER : null));
+    final String resourceDescription = "MemorySegmentIndexInput(path=\"" + path.toString() + "\")";
+    final long fileSize = Files.size(path);
+
+    boolean success = false;
+    final ResourceScope scope = ResourceScope.newSharedScope();
+    try {
+      final MemorySegment[] segments = map(scope, resourceDescription, path, fileSize);
+      final IndexInput in =
+          MemorySegmentIndexInput.newInstance(
+              resourceDescription, scope, segments, fileSize, chunkSizePower);
+      success = true;
+      return in;

Review comment:
       It's closed here: https://github.com/apache/lucene/pull/177/files/6d16a8d0984c73ddb32fced02a180b1a688662c1#diff-1c8534e50633cd8648f2cb3c886a269a6ebf8a57d817885bd962225d83f4e93fR456




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

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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866013290


   There is no fix for the JDK 17 issues with garbage collection and disabled tiered compilation. From Java 18 on we can use the new MemoryCopy class to do bulk copies. It also supports byte swap to adjust endianness: https://github.com/openjdk/panama-foreign/pull/555
   
   The results look promising. I have a branch using that available already, will create a new pull request to record the iterations.
   
   Still we need to investiage if for the readLongs() and readFloats() the loop is better, as the array sizes are really small and the bulk copy overhead is large. I can see t already in the benchmarks.


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

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



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


[GitHub] [lucene] jbhateja commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
jbhateja commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891951566


   > It works with build 33, every hour a Jenkins build tests it: https://jenkins.thetaphi.de/job/Lucene-jdk17panama-Linux/ and https://jenkins.thetaphi.de/job/Lucene-jdk17panama-Windows/
   > 
   > Before 25 the API of Panama was on state of java 16, in that case you need to use the Java 16 pull request.
   > 
   > In addition make sure to _not_ use any system installed Gradle but `./gradlew`.
   
   Thanks @uschindler for your help, I have removed my other comments from this patch to reduce the noise.


-- 
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] jbhateja removed a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
jbhateja removed a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-890407948


   Hi @uschindler ,
   I tried to build your patch but get the following error. Can you kindly suggest what could be the issue.
   
   Error log:-
   
   > Task :altJvmWarning
   NOTE: Alternative java toolchain will be used for compilation and tests:
     Project will use Java 17 from: /mnt/c/Users/jatin/home-ubuntu/softwares/jdk-17
     Gradle runs with Java 14 from: /usr/lib/jvm/java-14-openjdk-amd64
   
   
   > Task :errorProneSkipped
   WARNING: errorprone disabled (skipped on non-nightly runs)
   
   > Task :lucene:core:compileJava
   warning: using incubating module(s): jdk.incubator.foreign
   /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:27: error: cannot find symbol
   import jdk.incubator.foreign.ResourceScope;
                               ^
     symbol:   class ResourceScope
     location: package jdk.incubator.foreign
   /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:51: error: cannot find symbol
     final ResourceScope scope;
           ^
     symbol:   class ResourceScope
   
   Thanks in advance.


-- 
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 #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       It is called "unwrapable" (I can fix the typo), because this interface is implemented by all classes that can unwrap themselves (like FilterPath can return its inner Path).




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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {
+      return alreadyClosed();
+    }
+    // TODO: maybe open a JDK issue to have a separate, more
+    // meaningful exception for unmapped segments:
+    if (e.getMessage() != null && e.getMessage().contains("closed")) {
+      return alreadyClosed();
+    }
+    return e;
+  }
+
+  RuntimeException handlePositionalIOOBE(String action, long pos) throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position: " + this);
+    } else {
+      throw new EOFException(action + " past EOF: " + this);
+    }
+  }
+
+  private AlreadyClosedException alreadyClosed() {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      do {
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+      } while (curSegment.byteSize() == 0L);
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    final MemorySegment arraySegment = MemorySegment.ofArray(b);
+    try {
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(arraySegment, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  private void readBytesBoundary(final MemorySegment arraySegment, int offset, int len)
+      throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, curAvail));
+        len -= curAvail;
+        offset += curAvail;
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+        curAvail = curSegment.byteSize();
+      }
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public void readLongs(long[] dst, int offset, int length) throws IOException {
+    if (IS_LITTLE_ENDIAN) {
+      final MemorySegment targetSlice =
+          MemorySegment.ofArray(dst).asSlice((long) offset << 3, (long) length << 3);
+      try {
+        targetSlice.copyFrom(curSegment.asSlice(curPosition, targetSlice.byteSize()));
+        curPosition += targetSlice.byteSize();
+      } catch (
+          @SuppressWarnings("unused")
+          IndexOutOfBoundsException iobe) {
+        super.readLongs(dst, offset, length);

Review comment:
       If you are afraid that this is not tested. Don't worry: The MultiMMap test case creates MMapDirectory instances with very small chunk sizes (small as a segments of 32 bytes) and tests all methods - inherited from ByteBuffer MMAP. Any problem in the catch code would break test :-)




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

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



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


[GitHub] [lucene] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866784477


   > > > The problem with luceneutil is also that it respawns a JVM multiple times.
   > > 
   > > 
   > > Hmm, we added multiple JVMs long ago precisely because HotSpot was so unpredictable. I.e. we had clear examples where HotSpot would paint itself into a corner, compiling e.g. `readVInt` poorly and never re-compiling it, or something, such that no matter how long the benchmark ran, it would never reach as good performance as if you simply restarted the whole JVM and rolled the dice again. But maybe this situation has been improved and these were somehow early HotSpot bugs/issues and we could really remove multiple JVMs without harming how accurately we can extract the mean/variance performance of all our benchmark tasks?
   > 
   > This is also not reality: Would you restart your Elasticsearch server from time to time because you think there might be a broken `readVInt()` optimization?
   
   Yeah, that is true!  But perhaps it shouldn't be the case :)  Maybe Elasticsearch/OpenSearch/Solr should spawn JVM a few times until they get a "good" `readVInt` compilation!  The noisy mis-compilation was such a sizable impact (back then, hopefully not anymore?).
   
   If we only ran benchmarks in nightly runs so that we could see that noise/variance with time, maybe we could do just one JVM.
   
   But when a developer is trying to test an exciting optimization, in the privacy of their `git clone`, it really sucks to have hotspot noise completely drown out any small gains your optimization might show!
   
   Benchmarking is hard :)
   
   > Here are the berlinbuzzwords slides about this: https://2021.berlinbuzzwords.de/sites/berlinbuzzwords.de/files/2021-06/The%20future%20of%20Lucene%27s%20MMapDirectory.pdf
   
   Oooh, thanks for sharing!  The talk looks AWESOME!  I will watch recording when it's out :)  You should share these slides on Twitter too?


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860731308


   I used the wrong benchmark task (not wikimediumall but wikimedium10k, copypastershitnwaste). New benchmarks coming soon.


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860900058


   With my patch to bulk methods results look like this:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
                    Respell       55.14      (0.9%)       40.97      (0.9%)  -25.7% ( -27% -  -24%) 0.000
                     Fuzzy1       66.39      (6.3%)       51.40      (2.7%)  -22.6% ( -29% -  -14%) 0.000
                     Fuzzy2       63.82      (6.7%)       50.99      (2.3%)  -20.1% ( -27% -  -11%) 0.000
                   PKLookup      190.65      (1.6%)      172.48      (2.1%)   -9.5% ( -13% -   -5%) 0.000
                   Wildcard       12.84      (6.3%)       11.62      (4.0%)   -9.5% ( -18% -    0%) 0.004
          HighTermMonthSort       61.20     (19.6%)       56.61     (15.7%)   -7.5% ( -35% -   34%) 0.505
           HighSloppyPhrase        1.93      (5.5%)        1.80     (10.8%)   -6.8% ( -21% -   10%) 0.213
            MedSloppyPhrase        8.01      (3.2%)        7.47      (8.4%)   -6.6% ( -17% -    5%) 0.099
            LowSloppyPhrase       11.52      (2.7%)       10.79      (5.4%)   -6.3% ( -14% -    1%) 0.020
       HighTermTitleBDVSort       76.19      (8.5%)       72.67      (9.6%)   -4.6% ( -20% -   14%) 0.421
      HighTermDayOfYearSort       59.52      (7.0%)       57.12      (7.7%)   -4.0% ( -17% -   11%) 0.386
                    Prefix3       67.21     (13.5%)       64.93     (14.0%)   -3.4% ( -27% -   27%) 0.696
                 TermDTSort       58.93     (12.0%)       56.98     (15.9%)   -3.3% ( -27% -   27%) 0.711
                MedSpanNear        5.41      (1.5%)        5.24      (1.3%)   -3.1% (  -5% -    0%) 0.001
                LowSpanNear       21.34      (0.6%)       20.72      (1.2%)   -2.9% (  -4% -   -1%) 0.000
                 OrHighHigh       15.81      (1.1%)       15.35      (2.5%)   -2.9% (  -6% -    0%) 0.019
      BrowseMonthSSDVFacets        4.29      (2.8%)        4.16      (4.1%)   -2.8% (  -9% -    4%) 0.201
                  OrHighMed       18.71      (1.6%)       18.19      (2.7%)   -2.8% (  -6% -    1%) 0.046
                  LowPhrase      139.63      (1.9%)      136.16      (1.3%)   -2.5% (  -5% -    0%) 0.018
                  MedPhrase      114.24      (2.2%)      111.48      (1.0%)   -2.4% (  -5% -    0%) 0.025
                 HighPhrase      180.83      (2.5%)      176.61      (1.4%)   -2.3% (  -6% -    1%) 0.065
                 AndHighLow      496.29      (4.3%)      484.90      (2.4%)   -2.3% (  -8% -    4%) 0.300
               HighSpanNear        9.11      (2.4%)        8.94      (2.5%)   -1.9% (  -6% -    3%) 0.233
   BrowseDayOfYearSSDVFacets        4.14      (2.5%)        4.08      (0.4%)   -1.5% (  -4% -    1%) 0.172
                     IntNRQ       38.03      (0.6%)       37.51      (1.4%)   -1.4% (  -3% -    0%) 0.047
               OrNotHighLow      626.89      (4.7%)      618.44      (5.3%)   -1.3% ( -10% -    9%) 0.669
                  OrHighLow      280.08      (2.5%)      276.64      (3.2%)   -1.2% (  -6% -    4%) 0.494
                AndHighHigh       30.63      (2.5%)       30.30      (2.3%)   -1.1% (  -5% -    3%) 0.468
                 AndHighMed       35.60      (2.1%)       35.23      (1.7%)   -1.0% (  -4% -    2%) 0.385
                    MedTerm     1212.11      (3.4%)     1204.02      (6.3%)   -0.7% ( -10% -    9%) 0.836
       HighIntervalsOrdered       10.38      (3.7%)       10.42      (5.5%)    0.4% (  -8% -    9%) 0.880
               OrHighNotLow      738.69      (3.0%)      744.71      (7.0%)    0.8% (  -8% -   11%) 0.811
                    LowTerm     1563.73      (2.3%)     1577.33      (4.4%)    0.9% (  -5% -    7%) 0.696
              OrHighNotHigh      725.65      (3.8%)      732.67      (6.1%)    1.0% (  -8% -   11%) 0.763
               OrNotHighMed      681.22      (1.2%)      688.45      (4.5%)    1.1% (  -4% -    6%) 0.609
               OrHighNotMed      672.06      (3.8%)      679.93      (6.9%)    1.2% (  -9% -   12%) 0.741
                   HighTerm     1615.37      (4.1%)     1636.17      (4.2%)    1.3% (  -6% -    9%) 0.623
              OrNotHighHigh      634.61      (2.5%)      645.91      (7.2%)    1.8% (  -7% -   11%) 0.600
   BrowseDayOfYearTaxoFacets        0.99      (5.4%)        1.21      (7.8%)   21.8% (   8% -   37%) 0.000
       BrowseDateTaxoFacets        0.99      (5.2%)        1.21      (7.9%)   22.0% (   8% -   37%) 0.000
      BrowseMonthTaxoFacets        1.02      (4.1%)        1.25      (6.3%)   22.8% (  11% -   34%) 0.000
   ```
   
   No idea why this happens.


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

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



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


[GitHub] [lucene] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866787723


   > the JMH will do this too. I forget the defaults, but uses multiple jvm iterations and iterations within each jvm and warmup iterations. But it has smarts around the JIT compiler and can dump profiled assembly for its microbenchmarks. I never have noise issues with it.
   
   Excellent!
   
   > The current big "integration test" (lucene util) is useful for some things: e.g. something has to tell us there is pollution from too many java abstractions going megamorphic and so on :) 
   
   +1
   
   It really is more of an integration test, yeah.  It runs many different kinds of queries/tasks, concurrently across multiple threads, trying to exercise Lucene roughly in a way that OpenSearch/Elasticsearch/Solr might.  Though, it does not do concurrent indexing with searching in a single JVM, at least not with the default benchmarks.  Really, distributed search engines should not do that -- they should rather use [Lucene's near-real-time segment replication](https://blog.mikemccandless.com/2017/09/lucenes-near-real-time-segment-index.html#:~:text=Lucene's%20near%2Dreal%2Dtime%20segment%20index%20replication,-%5BTL%3BDR%3A&text=Lucene%20has%20a%20unique%20write,files%20will%20never%20again%20change.), which is more efficient if you have deep replicas, and also enables strong physical isolation of indexing and searching JVMs which have very different resources requirements!  OK `</soapbox>`!
   
   > But I think it would be improved by providing some more diagnostics (LogCompilation or whatever, maybe JIT stats in the JFR output). Let it be a "canary" to find little ways to improve.
   
   +1.  I wonder if we could tap into those in real-time and get a sense of when the JVM really is roughly "warmed up", instead of the static "discard first N samples for each task" that we do now.  Or maybe to detect mis-compilation of `readVInt`!
   
   > But we have nothing setup to do simple noise-free microbenchmarks over some specific code, e.g. like "unit tests" running different query types. And for those you don't want crazy JFR and logging and stuff as it is so targeted, you can just dump the hot assembly code instead. For now if you want to do this, you are writing one-off stuff yourself.
   
   Yeah maybe consing up a quick JMH for such cases is perfectly fine solution for we developers?


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866144962


   Here are the berlinbuzzwords slides about this: https://2021.berlinbuzzwords.de/sites/berlinbuzzwords.de/files/2021-06/The%20future%20of%20Lucene%27s%20MMapDirectory.pdf


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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       Sorry for not responding here. @dweiss talked about that already in the JDK-16 pull request (#173). It's a hack here, we can rename. I'd like to add those changes into Lucene before this PR is merged, so we can discuss this on a separate issue.




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

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



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


[GitHub] [lucene] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866073550


   > So in general the benchmark should mimic real-live.
   
   +1 -- nightly benchmarks already overrides this (poor) default, so let's fix `constants.py` to not pass `-Xbatch` nor `-XX:-TieredCompilation` 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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866054411


   Hi @mikemccand, 
   I will open a few more issues in luceneutil. The way how it invokes the JVM is not like it is done in production:
   - `-Xbatch` as default doesn't help for reproducibility and makes the results problematic for production use, as unrealistic.
   - `-XX:-TieredCompilation` as default is only useful to run tests (so we don't spend too much time for recompile), but as it is default in modern JDKS since Java 8, it should really be swritched on for benchmarks. Benchmarks run long time and they get real speed boosts by this. While testing this pull request, disabling tiered compilation made results 20% worse because it prevented escape analysis from kicking in early enough.
   
   With modern JVMs, doing batch compilation brings nothing, as a lot of stuff is optimized now also on the Java side, not just in Hotspot. E.g., when a lambda or functional method reference is used with stream() API, they are also optimized at runtime outside hotspot by rewriting it to temporary classes and optimized byte code (lambda trasformations, invokedynamic). 
   
   So in general the benchmark should mimic real-live. You are repating the runs several times anyways, so changing the optimizations to be non-default is not a good idea, because your settings do not have less fluctuation don't help, they slow down!


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

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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861492558


   The issue is confirmed and for the readBytes() code there's already a workaround. Long term we will improve to have this for all array types. See discussion here: https://github.com/openjdk/panama-foreign/pull/560
   
   For the float and long options:  copyMemory() has some overhead for small array sizes, so the recommendation by Panama and Hotspot engineers is to remove the specialization from Lucene at all. readLongs() will only read a maximum of 64 longs, a loop with removed bounds checks will do much better here than an explicit memory copy.
   
   For vectors I have the same feeling, but there we should go in direction of removing readFloats() alltogether and replace by a method returning a FloatVector() view 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



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


[GitHub] [lucene] markrmiller commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
markrmiller commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-935366867


   >  and also enables strong physical isolation of indexing and searching JVMs which have very different resources requirements!  OK
   
   This is a real issue with impact that comes up. Indexing on the replica has a surprising impact on query performance if you believe the reports. Those used to adding a bunch of replicas just for read side fan out felt the difference if they tried to migrate.


-- 
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 pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861046986


   I did more testig and was able to make MMapDirectory work the same speed. This path shows the problems: https://github.com/uschindler/lucene/commit/b057213cc6548ffa29b8e2810f39eb84c50a3bc0
   
   The main issue can be seen in the copying still used by Lucene between mmaped segment and heap:
   ```java
     private static void copySegmentToHeap(MemorySegment src, long srcOffset, byte[] target, int targetOffset, int len) {
       Objects.checkFromIndexSize(srcOffset, len, src.byteSize());
       theUnsafe.copyMemory(null, src.address().toRawLongValue() + srcOffset, 
           target, Unsafe.ARRAY_BYTE_BASE_OFFSET + targetOffset, len);
       //MemorySegment.ofArray(target).asSlice(targetOffset, len).copyFrom(src.asSlice(srcOffset, len));
     }
   ```
   
   If you do the memory copy using Unsafe as this static method shows, the performance is identical:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
                   PKLookup      186.78      (2.2%)      180.11      (1.4%)   -3.6% (  -7% -    0%) 0.000
      BrowseMonthSSDVFacets        4.14      (6.0%)        4.05      (6.3%)   -2.3% ( -13% -   10%) 0.238
   BrowseDayOfYearSSDVFacets        4.03      (4.4%)        3.94      (4.7%)   -2.2% ( -10% -    7%) 0.123
                    Prefix3        5.43     (10.4%)        5.36     (11.7%)   -1.3% ( -21% -   23%) 0.708
                     IntNRQ       41.13      (2.0%)       40.61      (1.8%)   -1.2% (  -4% -    2%) 0.040
            LowSloppyPhrase        5.85      (4.2%)        5.79      (3.8%)   -1.0% (  -8% -    7%) 0.411
           HighSloppyPhrase        9.74      (4.1%)        9.64      (3.9%)   -1.0% (  -8% -    7%) 0.416
                 HighPhrase      226.76      (2.7%)      225.13      (2.9%)   -0.7% (  -6% -    5%) 0.417
                  MedPhrase      204.52      (2.7%)      203.77      (2.5%)   -0.4% (  -5% -    4%) 0.656
                     Fuzzy1       84.24      (4.5%)       84.20      (5.9%)   -0.0% (  -9% -   10%) 0.979
                  LowPhrase       23.33      (2.6%)       23.33      (2.7%)   -0.0% (  -5% -    5%) 0.986
                    Respell       34.95      (1.5%)       34.97      (1.6%)    0.1% (  -3% -    3%) 0.898
                   HighTerm     1128.45      (4.8%)     1129.85      (4.7%)    0.1% (  -8% -   10%) 0.934
          HighTermMonthSort       86.97     (17.0%)       87.24     (14.5%)    0.3% ( -26% -   38%) 0.951
               OrHighNotLow      913.63      (4.0%)      919.28      (3.5%)    0.6% (  -6% -    8%) 0.603
               OrNotHighLow      630.14      (2.8%)      635.01      (3.0%)    0.8% (  -4% -    6%) 0.401
                AndHighHigh       42.87      (4.2%)       43.22      (4.4%)    0.8% (  -7% -    9%) 0.550
                    MedTerm     1504.76      (4.3%)     1519.72      (3.8%)    1.0% (  -6% -    9%) 0.440
            MedSloppyPhrase       38.70      (3.2%)       39.11      (2.6%)    1.1% (  -4% -    7%) 0.251
                 AndHighLow      464.18      (4.4%)      469.92      (4.6%)    1.2% (  -7% -   10%) 0.388
               OrNotHighMed      762.97      (2.8%)      773.13      (3.2%)    1.3% (  -4% -    7%) 0.162
                  OrHighLow      290.51      (4.7%)      294.49      (4.3%)    1.4% (  -7% -   10%) 0.339
                     Fuzzy2       51.19      (8.4%)       51.90      (7.2%)    1.4% ( -13% -   18%) 0.576
                    LowTerm     1892.07      (2.9%)     1919.02      (4.0%)    1.4% (  -5% -    8%) 0.199
                 AndHighMed       43.09      (5.0%)       43.76      (4.7%)    1.6% (  -7% -   11%) 0.311
               OrHighNotMed      708.72      (3.0%)      719.93      (2.7%)    1.6% (  -4% -    7%) 0.084
              OrHighNotHigh      791.29      (4.2%)      803.91      (3.7%)    1.6% (  -6% -    9%) 0.204
                  OrHighMed       37.87      (4.8%)       38.52      (3.2%)    1.7% (  -5% -   10%) 0.179
              OrNotHighHigh      755.35      (3.4%)      769.72      (3.6%)    1.9% (  -4% -    9%) 0.084
      HighTermDayOfYearSort       66.49     (17.2%)       67.81     (15.3%)    2.0% ( -25% -   41%) 0.700
       HighIntervalsOrdered        5.10      (4.0%)        5.20      (3.9%)    2.0% (  -5% -   10%) 0.108
       HighTermTitleBDVSort       71.78     (17.6%)       73.23     (16.8%)    2.0% ( -27% -   44%) 0.712
                 OrHighHigh        9.62      (6.7%)        9.83      (3.2%)    2.2% (  -7% -   13%) 0.181
                   Wildcard       13.65      (9.0%)       14.11     (10.0%)    3.4% ( -14% -   24%) 0.264
                 TermDTSort       61.68     (15.7%)       64.16      (9.1%)    4.0% ( -18% -   34%) 0.324
               HighSpanNear        8.32      (2.6%)        8.72      (3.1%)    4.7% (   0% -   10%) 0.000
                LowSpanNear       10.98      (2.1%)       11.54      (2.2%)    5.1% (   0% -    9%) 0.000
                MedSpanNear        8.34      (2.0%)        8.77      (2.2%)    5.2% (   0% -    9%) 0.000
      BrowseMonthTaxoFacets        1.02      (3.5%)        1.13     (13.6%)   10.1% (  -6% -   28%) 0.001
   BrowseDayOfYearTaxoFacets        1.00      (4.6%)        1.11     (14.4%)   10.5% (  -8% -   30%) 0.002
       BrowseDateTaxoFacets        1.00      (4.6%)        1.11     (14.4%)   10.7% (  -8% -   31%) 0.002
    
   CPU merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_
 version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/bench
 mark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 4.04 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1544656 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   15.33%        236729        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.95%         153617        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   7.58%         117056        org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   5.42%         83667         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   4.47%         68989         jdk.internal.misc.ScopedMemoryAccess#getShortUnalignedInternal()
   4.43%         68353         jdk.internal.foreign.Utils#filterSegment()
   2.49%         38393         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   1.98%         30594         jdk.internal.foreign.AbstractMemorySegmentImpl#checkBoundsSmall()
   1.91%         29460         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.82%         28104         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.73%         26738         jdk.internal.foreign.AbstractMemorySegmentImpl#isSet()
   1.72%         26516         sun.misc.Unsafe#copyMemory()
   1.63%         25197         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#nextDoc()
   1.55%         23880         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
   1.42%         21908         jdk.internal.util.Preconditions#checkFromIndexSize()
   1.31%         20268         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   1.10%         16954         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   1.00%         15398         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.94%         14473         java.lang.invoke.VarHandleGuards#guard_LJ_I()
   0.90%         13916         jdk.internal.foreign.SharedScope#checkValidState()
   0.83%         12820         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.78%         12064         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.73%         11224         org.apache.lucene.index.SingletonSortedSetDocValues#nextOrd()
   0.72%         11055         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.71%         10950         org.apache.lucene.store.DataInput#readVInt()
   0.69%         10643         org.apache.lucene.store.MemorySegmentIndexInput$SingleSegmentImpl#seek()
   0.69%         10612         org.apache.lucene.queries.intervals.OrderedIntervalsSource$OrderedIntervalIterator#nextInterval()
   0.61%         9380          jdk.internal.foreign.AbstractMemorySegmentImpl#scope()
   0.57%         8825          org.apache.lucene.util.BitSet#or()
   0.57%         8747          java.util.Objects#checkIndex()
    
    
   CPU merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=cpu -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-ba
 seline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 4.24 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 1712367 events (total: 1M)
     tests.profile.mode=cpu
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       CPU SAMPLES   STACK
   12.99%        222468        org.apache.lucene.util.packed.DirectMonotonicReader#get()
   9.56%         163732        org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts#countAll()
   5.78%         99057         java.nio.ByteBuffer#getArray()
   5.05%         86545         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$15#binaryValue()
   4.12%         70481         org.apache.lucene.util.packed.DirectReader$DirectPackedReader12#get()
   3.96%         67813         java.nio.Buffer#scope()
   3.87%         66312         jdk.internal.misc.Unsafe#convEndian()
   3.81%         65189         org.apache.lucene.store.ByteBufferGuard#ensureValid()
   3.48%         59539         org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts#countOneSegment()
   2.30%         39343         org.apache.lucene.store.ByteBufferGuard#getBytes()
   2.14%         36573         java.nio.ByteBuffer#get()
   1.88%         32278         org.apache.lucene.index.SingletonSortedSetDocValues#nextDoc()
   1.67%         28624         java.nio.Buffer#checkIndex()
   1.53%         26222         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#nextDoc()
   1.22%         20948         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#advance()
   1.19%         20403         org.apache.lucene.store.ByteBufferGuard#getShort()
   1.03%         17553         org.apache.lucene.facet.taxonomy.IntTaxonomyFacets#increment()
   1.01%         17218         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#advance()
   0.94%         16167         java.nio.Buffer#position()
   0.89%         15218         org.apache.lucene.store.ByteBufferIndexInput#readBytes()
   0.88%         15123         org.apache.lucene.search.ConjunctionDISI#doNext()
   0.76%         12938         org.apache.lucene.store.ByteBufferIndexInput$SingleBufferImpl#seek()
   0.75%         12878         org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer$19#ordValue()
   0.75%         12813         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#findFirstGreater()
   0.73%         12537         org.apache.lucene.util.packed.DirectReader$DirectPackedReader4#get()
   0.69%         11839         org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$EverythingEnum#nextPosition()
   0.69%         11738         org.apache.lucene.store.DataInput#readVInt()
   0.65%         11070         org.apache.lucene.queries.intervals.OrderedIntervalsSource$OrderedIntervalIterator#nextInterval()
   0.62%         10575         jdk.internal.util.Preconditions#checkFromIndexSize()
   0.59%         10022         jdk.internal.misc.ScopedMemoryAccess#getByteInternal()
    
    
   HEAP merged search profile for my_modified_version:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_candidate/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified
 _version-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-6.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-7.jfr /home/thetaphi/benc
 hmark/util/bench-search-baseline_vs_patch-my_modified_version-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-my_modified_version-3.jfr
   Took 2.67 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 68380 events (total: 25828M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   18.44%        4761M         org.apache.lucene.util.FixedBitSet#<init>()
   7.55%         1948M         java.util.AbstractList#iterator()
   5.46%         1409M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   5.21%         1345M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.55%         1174M         org.apache.lucene.util.BytesRef#<init>()
   4.14%         1068M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   3.95%         1020M         org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   3.75%         967M          org.apache.lucene.util.ArrayUtil#growExact()
   3.13%         808M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   2.90%         749M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.07%         534M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   1.90%         492M          java.util.ArrayList#grow()
   1.78%         460M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.43%         369M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.32%         340M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.31%         337M          java.util.AbstractList#listIterator()
   1.27%         328M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.23%         317M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.13%         290M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   1.11%         287M          java.util.ArrayList#iterator()
   1.09%         282M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.07%         276M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.06%         274M          org.apache.lucene.store.MemorySegmentIndexInput#buildSlice()
   1.01%         259M          java.util.Arrays#asList()
   0.97%         250M          java.util.Arrays#copyOf()
   0.96%         248M          org.apache.lucene.util.PriorityQueue#<init>()
   0.91%         233M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.87%         223M          org.apache.lucene.queryparser.classic.Token#newToken()
   0.86%         222M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.84%         216M          jdk.internal.foreign.MappedMemorySegmentImpl#dup()
    
    
   HEAP merged search profile for baseline:
   JFR aggregation command: /home/jenkins/tools/java/64bit/latest-jdk17/bin/java --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError -Xbatch -cp /home/thetaphi/benchmark/lucene_baseline/buildSrc/build/classes/java/main -Dtests.profile.mode=heap -Dtests.profile.stacksize=1 -Dtests.profile.count=30 org.apache.lucene.gradle.ProfileResults /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-3.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-17.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-16.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-13.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-12.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-5.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-18.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-b
 aseline-9.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-1.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-19.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-4.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-2.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-8.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-11.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-10.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-14.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-15.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-0.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-7.jfr /home/thetaphi/benchmark/util/bench-search-baseline_vs_patch-baseline-6.jfr
   Took 2.58 seconds
   WARNING: Using incubator modules: jdk.incubator.foreign
   PROFILE SUMMARY from 69795 events (total: 26355M)
     tests.profile.mode=heap
     tests.profile.count=30
     tests.profile.stacksize=1
     tests.profile.linenumbers=false
   PERCENT       HEAP SAMPLES  STACK
   17.98%        4739M         org.apache.lucene.util.FixedBitSet#<init>()
   7.47%         1968M         java.util.AbstractList#iterator()
   5.16%         1359M         org.apache.lucene.search.ExactPhraseMatcher$1$1#getImpacts()
   4.97%         1309M         org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnumFrame#<init>()
   4.44%         1169M         org.apache.lucene.util.BytesRef#<init>()
   4.08%         1074M         org.apache.lucene.search.ExactPhraseMatcher$1#getImpacts()
   3.91%         1030M         org.apache.lucene.util.fst.ByteSequenceOutputs#read()
   3.38%         892M          org.apache.lucene.queryparser.charstream.FastCharStream#refill()
   3.36%         885M          org.apache.lucene.util.ArrayUtil#growExact()
   2.81%         741M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockDocsEnum#<init>()
   2.08%         548M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#load()
   2.00%         528M          java.util.ArrayList#grow()
   1.82%         479M          jdk.internal.misc.Unsafe#allocateUninitializedArray()
   1.58%         415M          java.nio.DirectByteBufferR#duplicate()
   1.43%         376M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#getFrame()
   1.41%         370M          java.util.AbstractList#listIterator()
   1.33%         350M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader#newTermState()
   1.26%         330M          org.apache.lucene.queryparser.charstream.FastCharStream#GetImage()
   1.17%         307M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsDocsEnum#<init>()
   1.16%         306M          org.apache.lucene.codecs.lucene90.ForUtil#<init>()
   1.14%         299M          java.util.ArrayList#iterator()
   1.12%         295M          org.apache.lucene.codecs.lucene90.blocktree.SegmentTermsEnum#<init>()
   1.00%         263M          java.nio.DirectByteBufferR#slice()
   0.98%         257M          java.util.Arrays#asList()
   0.97%         254M          org.apache.lucene.util.PriorityQueue#<init>()
   0.91%         239M          org.apache.lucene.codecs.lucene90.Lucene90PostingsReader$BlockImpactsPostingsEnum#<init>()
   0.88%         231M          org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader$BlockState#document()
   0.73%         192M          java.nio.DirectByteBufferR#asLongBuffer()
   0.70%         184M          org.apache.lucene.codecs.lucene90.blocktree.IntersectTermsEnumFrame#<init>()
   0.68%         179M          org.apache.lucene.store.ByteBufferIndexInput#newCloneInstance()
   ```
   
   If you comment out the unsafe code and use the "official MemorySegmen API", the whole thing gets crazy:
   - runtime of a bench run on my machine oes up from 57 seconds to 75 seconds, while otheriwse (Unsafe) staying identical to baseline
   - in addition it produces a lot of garbage, heap dump contains many `HeapMemorySegmentImpl$OfByte` classes (that are the wrappers around `byte[]` when viewed as MemorySegment. Every wrapping produces a new instance which is not catched by escape analysis. This slows down! The heap dump has 50% of heap filled with those objects according to JFR.
   
   I will report this problem to project Panama! @mcimadamore 


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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {

Review comment:
       Thats the same in MMapDirectory using ByteBuffers.
   
   We don't do any checks for NULL or OFFSETS anywhere and just catch the exception. The actual check is too expensive. 
   
   The NullPointerException happens when the IndexInput is closed. We null out the segment on close. Adding some "if stillOpen" is not wanted, as a check on every access is way too expensive. See also Robert's comment 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



---------------------------------------------------------------------
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 #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {
+      return alreadyClosed();
+    }
+    // TODO: maybe open a JDK issue to have a separate, more
+    // meaningful exception for unmapped segments:
+    if (e.getMessage() != null && e.getMessage().contains("closed")) {
+      return alreadyClosed();
+    }
+    return e;
+  }
+
+  RuntimeException handlePositionalIOOBE(String action, long pos) throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position: " + this);
+    } else {
+      throw new EOFException(action + " past EOF: " + this);
+    }
+  }
+
+  private AlreadyClosedException alreadyClosed() {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      do {
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+      } while (curSegment.byteSize() == 0L);
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    final MemorySegment arraySegment = MemorySegment.ofArray(b);
+    try {
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(arraySegment, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  private void readBytesBoundary(final MemorySegment arraySegment, int offset, int len)
+      throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, curAvail));
+        len -= curAvail;
+        offset += curAvail;
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+        curAvail = curSegment.byteSize();
+      }
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public void readLongs(long[] dst, int offset, int length) throws IOException {
+    if (IS_LITTLE_ENDIAN) {
+      final MemorySegment targetSlice =
+          MemorySegment.ofArray(dst).asSlice((long) offset << 3, (long) length << 3);
+      try {
+        targetSlice.copyFrom(curSegment.asSlice(curPosition, targetSlice.byteSize()));
+        curPosition += targetSlice.byteSize();
+      } catch (
+          @SuppressWarnings("unused")
+          IndexOutOfBoundsException iobe) {
+        super.readLongs(dst, offset, length);

Review comment:
       In general we want to have only strictly needed checks in the main code path. If something goes wrong (e.g. we are at end of segment), we catch Exception and then go slow path.
   
   This is identical in ByteBufferDirectory used by the current MMapDirectory.




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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       I don't really care. For this type of interface I prefer the -able variant, because it is a single method interface just implementing one special ability, that can be used by testing for interface and then calling+casting.




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

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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866054411


   Hi @mikemccand, 
   I will open a few more issues in luceneutil. The way how it invokes the JVM is not like it is done in production:
   - `-Xbatch` as default doesn't help for reproducibility and makes the results problematic for production use, as unrealistic.
   - `-XX:-TieredCompilation` as default is only useful to run tests (so we don't spend too much time for recompile), but as it is default in modern JDKS since Java 8, it should really be swritched on for benchmarks. Benchmarks run long time and they get real speed boosts by this. While testing this pull request, disabling tiered compilation made results 20% worse because it prevented escape analysis from kicking in early enough.
   
   With modern JVMs, doing batch compilation brings nothing, as a lot of stuff is optimized now also on the Java side, not just in Hotspot. E.g., when a lambda or functional method reference is used with stream() API, they are also optimized at runtime outside hotspot by rewriting it to temporary classes and optimized byte code (lambda trasformations, invokedynamic). 
   
   So in general the benchmark should mimic real-live. You are repeating the runs several times anyways, so changing the optimizations to be non-default is not a good idea, because your settings do not have less fluctuation don't help, they slow down!


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866142054


   > > The problem with luceneutil is also that it respawns a JVM multiple times.
   > 
   > Hmm, we added multiple JVMs long ago precisely because HotSpot was so unpredictable. I.e. we had clear examples where HotSpot would paint itself into a corner, compiling e.g. `readVInt` poorly and never re-compiling it, or something, such that no matter how long the benchmark ran, it would never reach as good performance as if you simply restarted the whole JVM and rolled the dice again. But maybe this situation has been improved and these were somehow early HotSpot bugs/issues and we could really remove multiple JVMs without harming how accurately we can extract the mean/variance performance of all our benchmark tasks?
   
   This is also not reality: Would you restart your Elasticsearch server from time to time because you think there might be a broken `readVInt()` optimization?
   
   


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866112156


   > > This was negotiated today 👍
   > > To read the whole story check here: [openjdk/panama-foreign#555 (comment)](https://github.com/openjdk/panama-foreign/pull/555#issuecomment-865672909)
   > 
   > Wow, awesome! I love that your efforts to get Lucene working well on top of these new Java-friendly `mmap` APIs is uncovering things to fix / iterations to iterate, thanks to unexpectedly slower Lucene faceting!
   > 
   > `luceneutil` throws away first few iterations of each task to allow for "warmup", but I guess it was not enough here.
   > 
   > Maybe we could fix `luceneutil` to do this more dynamically, e.g. introspect on when hotspot has "mostly" finished compiling the hotspots, and things have reached steady state, instead of the fixed "discard first N results per task"?
   
   I think this is what JMH should help us to do. The problem with luceneutil is also that it respawns a JVM multiple times. Maybe it should just execute more queries (the same ones in a loop) in the same JVM (like running 20 times as long instead of 20 separate JVMs) and then drop like half of the measurements. The problem is that Hotspot is optimizing all the time and it's hard to figure out when it is 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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860641916


   Hi,
   I executed luceneutil bench using jdk-17-ea+b25:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
      BrowseMonthTaxoFacets     4136.71      (3.6%)     2684.53      (4.2%)  -35.1% ( -41% -  -28%) 0.000
       BrowseDateTaxoFacets     3842.73      (4.5%)     2579.73      (1.8%)  -32.9% ( -37% -  -27%) 0.000
   BrowseDayOfYearTaxoFacets     3269.90      (4.4%)     2296.66      (3.1%)  -29.8% ( -35% -  -23%) 0.000
                   PKLookup      340.50      (3.6%)      305.61      (1.9%)  -10.2% ( -15% -   -4%) 0.000
            MedSloppyPhrase      750.49      (9.8%)      684.83      (8.5%)   -8.7% ( -24% -   10%) 0.003
               HighSpanNear      758.20      (8.0%)      703.82     (11.1%)   -7.2% ( -24% -   12%) 0.019
       HighIntervalsOrdered      932.88      (7.4%)      872.75      (9.9%)   -6.4% ( -22% -   11%) 0.019
                     Fuzzy1      453.74     (13.8%)      431.35     (11.2%)   -4.9% ( -26% -   23%) 0.214
           HighSloppyPhrase      825.27     (11.2%)      784.94      (7.9%)   -4.9% ( -21% -   16%) 0.111
                AndHighHigh     1069.65     (11.0%)     1028.11     (10.5%)   -3.9% ( -22% -   19%) 0.254
                    Respell      365.08     (10.8%)      352.91      (9.1%)   -3.3% ( -20% -   18%) 0.291
                 AndHighLow     5517.43      (6.7%)     5340.85      (7.9%)   -3.2% ( -16% -   12%) 0.168
                MedSpanNear     1480.21      (9.8%)     1434.36      (6.9%)   -3.1% ( -18% -   15%) 0.249
                  LowPhrase     1073.64      (5.9%)     1042.16      (6.9%)   -2.9% ( -14% -   10%) 0.150
      HighTermDayOfYearSort     4737.54      (5.7%)     4619.94      (3.7%)   -2.5% ( -11% -    7%) 0.102
            LowSloppyPhrase     1402.06      (5.7%)     1367.43      (5.6%)   -2.5% ( -13% -    9%) 0.166
                  OrHighMed      894.25     (10.3%)      873.07      (9.5%)   -2.4% ( -20% -   19%) 0.450
                   Wildcard      576.83      (8.6%)      563.68      (6.4%)   -2.3% ( -15% -   13%) 0.343
                     Fuzzy2       59.21     (16.1%)       58.06     (18.1%)   -1.9% ( -31% -   38%) 0.719
                 HighPhrase      647.47      (8.7%)      635.63      (9.3%)   -1.8% ( -18% -   17%) 0.521
                    MedTerm     7892.79      (4.3%)     7751.00      (4.6%)   -1.8% ( -10% -    7%) 0.206
                   HighTerm     6971.95      (4.4%)     6867.18      (6.0%)   -1.5% ( -11% -    9%) 0.365
                LowSpanNear     1689.37      (8.7%)     1664.10      (7.8%)   -1.5% ( -16% -   16%) 0.565
          HighTermMonthSort     5044.03      (4.2%)     4970.03      (6.6%)   -1.5% ( -11% -    9%) 0.399
                     IntNRQ     1565.30      (5.4%)     1549.65      (2.4%)   -1.0% (  -8% -    7%) 0.451
                  MedPhrase      890.99      (9.5%)      884.98     (10.3%)   -0.7% ( -18% -   21%) 0.830
                    Prefix3      585.76     (16.3%)      588.03     (18.8%)    0.4% ( -29% -   42%) 0.944
                 AndHighMed     1946.50      (8.1%)     1954.42      (8.3%)    0.4% ( -14% -   18%) 0.875
   BrowseDayOfYearSSDVFacets     2349.67      (2.7%)     2360.57      (2.7%)    0.5% (  -4% -    5%) 0.583
                 OrHighHigh      402.33     (16.4%)      404.38     (13.1%)    0.5% ( -24% -   35%) 0.913
                  OrHighLow     1760.39     (11.1%)     1770.64      (8.4%)    0.6% ( -17% -   22%) 0.852
                    LowTerm     8072.83      (4.7%)     8140.49      (5.0%)    0.8% (  -8% -   11%) 0.586
      BrowseMonthSSDVFacets     2949.63      (6.6%)     3015.76      (3.2%)    2.2% (  -7% -   12%) 0.170
   ```
   
   For those interested:
   
   ```
   JAVA:
   WARNING: Using incubator modules: jdk.incubator.foreign
   openjdk version "17-ea" 2021-09-14
   OpenJDK Runtime Environment (build 17-ea+25-2252)
   OpenJDK 64-Bit Server VM (build 17-ea+25-2252, mixed mode, sharing)
   
   
   OS:
   Linux serv1.sd-datasolutions.de 5.8.0-50-generic #56~20.04.1-Ubuntu SMP Mon Apr 12 21:46:35 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
   
   
   LOGS:
   /home/thetaphi/benchmark/logs
   ```
   
   My localconstants.py, so that's how I added the module command line params. @mikemccand It would be better to have the tool respect `RUNTIME_JAVA_HOME` like gradlew does.
   
   ```python
   BASE_DIR = '/home/thetaphi/benchmark'
   BENCH_BASE_DIR = '/home/thetaphi/benchmark/util'
   
   JAVA_HOME = '/home/jenkins/tools/java/64bit/latest-jdk17'
   JAVA_EXE = '%s/bin/java' %JAVA_HOME
   JAVAC_EXE = '%s/bin/javac' %JAVA_HOME
   
   JAVA_COMMAND = '%s --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError
   ```
   
   Executed this with all defaults in `localrun.py`:
   ```
   $ python3 src/python/localrun.py -source wikimedium10k
   ```
   
   I have no idea why the facetting stuff at the beginning of the bench output is so badly behaving with MMapDirectory#v2 on top of project panama, I'll ignore this for now. Maybe @mikemccand has an idea! The rest looks perfectly fine to me from the performance (19 runs).


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-861492558


   The issue is confirmed and for the readBytes() code there's already a workaround. Long term we will improve
   
   For the float and long options:  copyMemory() has some overhead for small array sizes, so the recommendation by Panama and Hotspot engineers is to remove the specialization from Lucene at all. readLongs() will only read a maximum of 64 longs, a loop with removed bounds checks will do much better here than an explicit memory copy.
   
   For vectors I have the same feeling, but there we should go in direction of removing readFloats() alltogether and replace by a method returning a FloatVector() view 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



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


[GitHub] [lucene] uschindler edited a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler edited a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860641916


   Hi,
   I executed luceneutil bench using jdk-17-ea+b25:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
      BrowseMonthTaxoFacets        1.00      (4.8%)        0.71      (6.7%)  -29.1% ( -38% -  -18%) 0.000
   BrowseDayOfYearTaxoFacets        0.98      (5.9%)        0.70      (7.3%)  -28.1% ( -38% -  -15%) 0.000
       BrowseDateTaxoFacets        0.98      (5.9%)        0.70      (7.2%)  -28.1% ( -38% -  -15%) 0.000
                 AndHighLow      326.58      (6.1%)      288.18      (3.6%)  -11.8% ( -20% -   -2%) 0.000
                  OrHighLow      203.59      (6.2%)      182.35      (3.3%)  -10.4% ( -18% -    0%) 0.000
                 AndHighMed       51.56      (6.6%)       46.25      (4.8%)  -10.3% ( -20% -    1%) 0.000
                    Respell       35.89      (1.5%)       32.70      (1.8%)   -8.9% ( -12% -   -5%) 0.000
            LowSloppyPhrase       13.87      (3.4%)       12.77      (3.1%)   -7.9% ( -13% -   -1%) 0.000
                   PKLookup      189.60      (2.6%)      174.64      (2.7%)   -7.9% ( -12% -   -2%) 0.000
                LowSpanNear       20.18      (3.1%)       18.76      (2.2%)   -7.0% ( -11% -   -1%) 0.000
                     Fuzzy1       59.60      (6.5%)       55.88      (6.2%)   -6.2% ( -17% -    6%) 0.002
                     Fuzzy2       49.20      (9.0%)       46.35      (7.5%)   -5.8% ( -20% -   11%) 0.026
                AndHighHigh       28.28      (4.6%)       26.69      (3.6%)   -5.6% ( -13% -    2%) 0.000
                MedSpanNear       14.89      (3.4%)       14.16      (3.4%)   -4.9% ( -11% -    1%) 0.000
            MedSloppyPhrase       52.15      (4.9%)       49.89      (4.2%)   -4.3% ( -12% -    5%) 0.003
                  OrHighMed       40.28      (2.9%)       38.64      (2.7%)   -4.1% (  -9% -    1%) 0.000
                  MedPhrase      247.62      (3.3%)      238.13      (3.0%)   -3.8% (  -9% -    2%) 0.000
      HighTermDayOfYearSort       21.04     (16.5%)       20.33     (13.5%)   -3.4% ( -28% -   31%) 0.475
               OrNotHighLow      457.51      (4.1%)      442.74      (2.9%)   -3.2% (  -9% -    3%) 0.004
                 OrHighHigh        7.00      (3.0%)        6.78      (3.0%)   -3.0% (  -8% -    2%) 0.001
               HighSpanNear        1.77      (3.4%)        1.72      (3.2%)   -2.8% (  -9% -    3%) 0.006
                   Wildcard       55.88      (3.4%)       54.34      (3.2%)   -2.8% (  -9% -    4%) 0.009
      BrowseMonthSSDVFacets        4.18      (5.4%)        4.08      (5.2%)   -2.5% ( -12% -    8%) 0.140
           HighSloppyPhrase        6.16      (7.9%)        6.01      (6.0%)   -2.4% ( -15% -   12%) 0.278
                 TermDTSort       56.84     (19.7%)       55.64     (15.8%)   -2.1% ( -31% -   41%) 0.707
   BrowseDayOfYearSSDVFacets        4.06      (3.8%)        3.97      (4.0%)   -2.1% (  -9% -    5%) 0.096
                  LowPhrase       38.74      (3.7%)       37.95      (2.7%)   -2.1% (  -8% -    4%) 0.047
                    Prefix3       59.38      (5.9%)       58.16      (6.0%)   -2.0% ( -13% -   10%) 0.273
       HighIntervalsOrdered        5.89      (6.2%)        5.78      (5.1%)   -2.0% ( -12% -    9%) 0.274
       HighTermTitleBDVSort       34.99     (13.7%)       34.60     (13.1%)   -1.1% ( -24% -   29%) 0.795
          HighTermMonthSort       56.96     (16.7%)       57.28     (14.7%)    0.6% ( -26% -   38%) 0.910
               OrHighNotLow      550.54      (5.1%)      558.86      (5.0%)    1.5% (  -8% -   12%) 0.347
                     IntNRQ       33.75     (30.5%)       34.28     (29.5%)    1.6% ( -44% -   88%) 0.870
                   HighTerm     1141.71      (6.2%)     1163.18      (4.6%)    1.9% (  -8% -   13%) 0.273
                 HighPhrase       12.90      (3.9%)       13.15      (4.3%)    1.9% (  -6% -   10%) 0.136
              OrHighNotHigh      640.96      (6.4%)      665.45      (3.8%)    3.8% (  -5% -   14%) 0.022
               OrNotHighMed      692.81      (4.4%)      726.70      (4.6%)    4.9% (  -3% -   14%) 0.001
              OrNotHighHigh      698.56      (4.5%)      739.13      (3.7%)    5.8% (  -2% -   14%) 0.000
                    MedTerm     1594.10      (5.5%)     1700.62      (4.6%)    6.7% (  -3% -   17%) 0.000
                    LowTerm     1531.60      (5.2%)     1647.26      (4.6%)    7.6% (  -2% -   18%) 0.000
               OrHighNotMed      911.74      (5.3%)      990.26      (4.4%)    8.6% (  -1% -   19%) 0.000
   ```
   
   For those interested:
   
   ```
   JAVA:
   WARNING: Using incubator modules: jdk.incubator.foreign
   openjdk version "17-ea" 2021-09-14
   OpenJDK Runtime Environment (build 17-ea+25-2252)
   OpenJDK 64-Bit Server VM (build 17-ea+25-2252, mixed mode, sharing)
   
   
   OS:
   Linux serv1.sd-datasolutions.de 5.8.0-50-generic #56~20.04.1-Ubuntu SMP Mon Apr 12 21:46:35 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
   
   
   LOGS:
   /home/thetaphi/benchmark/logs
   ```
   
   My localconstants.py, so that's how I added the module command line params. @mikemccand It would be better to have the tool respect `RUNTIME_JAVA_HOME` like gradlew does.
   
   ```python
   BASE_DIR = '/home/thetaphi/benchmark'
   BENCH_BASE_DIR = '/home/thetaphi/benchmark/util'
   
   JAVA_HOME = '/home/jenkins/tools/java/64bit/latest-jdk17'
   JAVA_EXE = '%s/bin/java' %JAVA_HOME
   JAVAC_EXE = '%s/bin/javac' %JAVA_HOME
   
   JAVA_COMMAND = '%s --add-modules jdk.incubator.foreign -server -Xms2g -Xmx2g -XX:-TieredCompilation -XX:+HeapDumpOnOutOfMemoryError
   ```
   
   Executed this with all defaults in `localrun.py`:
   ```
   $ python3 src/python/localrun.py -source wikimediumall
   ```
   
   I have no idea why the facetting stuff at the beginning of the bench output is so badly behaving with MMapDirectory#v2 on top of project panama, I'll ignore this for now. Maybe @mikemccand has an idea! The rest looks perfectly fine to me from the performance (19 runs).


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

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



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


[GitHub] [lucene] rmuir commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {
+      return alreadyClosed();
+    }
+    // TODO: maybe open a JDK issue to have a separate, more
+    // meaningful exception for unmapped segments:
+    if (e.getMessage() != null && e.getMessage().contains("closed")) {
+      return alreadyClosed();
+    }
+    return e;
+  }
+
+  RuntimeException handlePositionalIOOBE(String action, long pos) throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position: " + this);
+    } else {
+      throw new EOFException(action + " past EOF: " + this);
+    }
+  }
+
+  private AlreadyClosedException alreadyClosed() {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      do {
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+      } while (curSegment.byteSize() == 0L);
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    final MemorySegment arraySegment = MemorySegment.ofArray(b);
+    try {
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(arraySegment, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  private void readBytesBoundary(final MemorySegment arraySegment, int offset, int len)
+      throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, curAvail));
+        len -= curAvail;
+        offset += curAvail;
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+        curAvail = curSegment.byteSize();
+      }
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public void readLongs(long[] dst, int offset, int length) throws IOException {
+    if (IS_LITTLE_ENDIAN) {
+      final MemorySegment targetSlice =
+          MemorySegment.ofArray(dst).asSlice((long) offset << 3, (long) length << 3);
+      try {
+        targetSlice.copyFrom(curSegment.asSlice(curPosition, targetSlice.byteSize()));
+        curPosition += targetSlice.byteSize();
+      } catch (
+          @SuppressWarnings("unused")
+          IndexOutOfBoundsException iobe) {
+        super.readLongs(dst, offset, length);

Review comment:
       you can go through the history of why it looks this way. It happens because it is a rare event (exceptional case) and speeds up the optimized case. it may only make e.g. one PFOR block crossing boundaries slightly slower, and won't slow entire query.




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

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



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


[GitHub] [lucene] uschindler removed a comment on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler removed a comment on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860649214






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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       Funny to read: https://ell.stackexchange.com/questions/34140/when-to-double-the-consonant-before-the-suffix-able




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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860665332


   > I will now download the JFR files created by the tool and analyze them locally.
   
   No info from JFR files :-(


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866967688


   > > Here are the berlinbuzzwords slides about this: https://2021.berlinbuzzwords.de/sites/berlinbuzzwords.de/files/2021-06/The%20future%20of%20Lucene%27s%20MMapDirectory.pdf
   > 
   > Oooh, thanks for sharing! The talk looks AWESOME! I will watch recording when it's out :) You should share these slides on Twitter too?
   
   I got he link yesterday by Nina, will post it later on twitter. I just had no time.
   
   >Yeah, that is true! But perhaps it shouldn't be the case :) Maybe Elasticsearch/OpenSearch/Solr should spawn JVM a few times until they get a "good" readVInt compilation! The noisy mis-compilation was such a sizable impact (back then, hopefully not anymore?).
   
   If that is still there, show dumps of assembly and I will for sure open a bug report. This should not happen. At least not with tiered compilation. If you use batch compilation, of course it could be problematic, because it can't "re-optimize" easily. It has to wait for a trap caused by a wrong assumption and switch to interpreter first before trying agin.


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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java
##########
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import jdk.incubator.foreign.MemoryHandles;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
+
+/**
+ * Base IndexInput implementation that uses an array of MemorySegments to represent a file.
+ *
+ * <p>For efficiency, this class requires that the segment size are a power-of-two (<code>
+ * chunkSizePower</code>).
+ */
+public abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  // We pass 1L as alignment, because currently Lucene file formats are heavy unaligned: :(
+  static final VarHandle VH_getByte =
+      MemoryHandles.varHandle(byte.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getShort =
+      MemoryHandles.varHandle(short.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getInt =
+      MemoryHandles.varHandle(int.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+  static final VarHandle VH_getLong =
+      MemoryHandles.varHandle(long.class, 1L, ByteOrder.LITTLE_ENDIAN).withInvokeExactBehavior();
+
+  static final boolean IS_LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+  final boolean isClone;
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final ResourceScope scope;
+  final MemorySegment[] segments;
+
+  int curSegmentIndex = -1;
+  MemorySegment
+      curSegment; // redundant for speed: segments[curSegmentIndex], also marker if closed!
+  long curPosition; // relative to curSegment, not globally
+
+  public static MemorySegmentIndexInput newInstance(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(
+          resourceDescription, scope, segments[0], length, chunkSizePower, false);
+    } else {
+      return new MultiSegmentImpl(
+          resourceDescription, scope, segments, 0, length, chunkSizePower, false);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      ResourceScope scope,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower,
+      boolean isClone) {
+    super(resourceDescription);
+    this.scope = scope;
+    this.segments = segments;
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(scope::equals);
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.isClone = isClone;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed();
+    }
+  }
+
+  RuntimeException wrapAlreadyClosedException(RuntimeException e) {
+    if (e instanceof NullPointerException) {
+      return alreadyClosed();
+    }
+    // TODO: maybe open a JDK issue to have a separate, more
+    // meaningful exception for unmapped segments:
+    if (e.getMessage() != null && e.getMessage().contains("closed")) {
+      return alreadyClosed();
+    }
+    return e;
+  }
+
+  RuntimeException handlePositionalIOOBE(String action, long pos) throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position: " + this);
+    } else {
+      throw new EOFException(action + " past EOF: " + this);
+    }
+  }
+
+  private AlreadyClosedException alreadyClosed() {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      do {
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+      } while (curSegment.byteSize() == 0L);
+      final byte v = (byte) VH_getByte.get(curSegment, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    final MemorySegment arraySegment = MemorySegment.ofArray(b);
+    try {
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(arraySegment, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw wrapAlreadyClosedException(e);
+    }
+  }
+
+  private void readBytesBoundary(final MemorySegment arraySegment, int offset, int len)
+      throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, curAvail));
+        len -= curAvail;
+        offset += curAvail;
+        curSegmentIndex++;
+        if (curSegmentIndex >= segments.length) {
+          throw new EOFException("read past EOF: " + this);
+        }
+        curSegment = segments[curSegmentIndex];
+        curPosition = 0L;
+        curAvail = curSegment.byteSize();
+      }
+      arraySegment.asSlice(offset).copyFrom(curSegment.asSlice(curPosition, len));

Review comment:
       Yes.




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

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



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


[GitHub] [lucene] dweiss commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       I think technically this is a decorator pattern. "Decorated.unwrap()" doesn't sound too bad to me either.




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

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



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


[GitHub] [lucene] uschindler commented on a change in pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

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



##########
File path: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
##########
@@ -16,28 +16,17 @@
  */
 package org.apache.lucene.store;
 
-import static java.lang.invoke.MethodHandles.*;
-import static java.lang.invoke.MethodType.methodType;
-
 import java.io.IOException;
-import java.lang.invoke.MethodHandle;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
-import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Locale;
-import java.util.Objects;
 import java.util.concurrent.Future;
-import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
+import jdk.incubator.foreign.MemorySegment;
+import jdk.incubator.foreign.ResourceScope;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Unwrapable;

Review comment:
       I changed the name to `Unwrappable`.




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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891655796


   It works with build 33, every hour a Jenkins build tests it: https://jenkins.thetaphi.de/job/Lucene-jdk17panama-Linux/ and https://jenkins.thetaphi.de/job/Lucene-jdk17panama-Windows/
   
   Before 25 the API of Panama was on state of java 16, in that case you need to use the Java 16 pull request.
   
   In addition make sure to *not* use any system installed Gradle but `./gradlew`.


-- 
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] mikemccand commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-866097083


   
   > This was negotiated today 👍
   > 
   > To read the whole story check here: [openjdk/panama-foreign#555 (comment)](https://github.com/openjdk/panama-foreign/pull/555#issuecomment-865672909)
   
   Wow, awesome!  I love that your efforts to get Lucene working well on top of these new Java-friendly `mmap` APIs is uncovering things to fix / iterations to iterate, thanks to unexpectedly slower Lucene faceting!
   
   `luceneutil` throws away first few iterations of each task to allow for "warmup", but I guess it was not enough here.
   
   Maybe we could fix `luceneutil` to do this more dynamically, e.g. introspect on when hotspot has "mostly" finished compiling the hotspots, and things have reached steady state, instead of the fixed "discard first N results per task"?


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

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



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


[GitHub] [lucene] uschindler commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
uschindler commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-860889941


   After analyzing the heap dumps provided by JFR, I was able to figure out what the problem is.
   
   Basically, all native, VarHandle backed methods are fast and optimize nice. But all "bulk" read methods are slow and produce a lot of garbage on heap:
   
   The problem is that we can't copy from a memory segment to a heap `byte[]` or heap `float[]` natively! So the whole wrapping with `MemorySegment.ofArray()`, applying offsets and length, copy memory makes the whole thing produce too much garbage, because it looks like Hotspot isn't able to remove the object allocation.
   
   I did a quick test:
   - I removed bulk `readLongs()` and bulk `readFloats()` from source code and let it fall through to the simple readLong/readFloat loop of DataInput and the slowdown suddely was going to zero!
   - I replaced the readBytes() code also by a simple  loop in the case the byte[] length to read is < 4096 bytes
   
   See this hack commit (not in this branch): https://github.com/uschindler/lucene/commit/9b328a74746a04351a99f33f82515122b06d5baa
   
   The speed is much better, but some tasks are slower now, but this is related because now stuff like copying or uncompressing stored fields may be slower partially.


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

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



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


[GitHub] [lucene] jbhateja commented on pull request #177: Initial rewrite of MMapDirectory for JDK-17 preview (incubating) Panama APIs (>= JDK-17-ea-b25)

Posted by GitBox <gi...@apache.org>.
jbhateja commented on pull request #177:
URL: https://github.com/apache/lucene/pull/177#issuecomment-891191240


   > > Hi @uschindler,
   > > I get the following compilation errors when I build your patch (Gradle build 6.3.8)
   > > > Task :altJvmWarning
   > > > NOTE: Alternative java toolchain will be used for compilation and tests:
   > > > Project will use Java 17 from: /mnt/c/Users/jatin/home-ubuntu/softwares/jdk-17
   > > > Gradle runs with Java 14 from: /usr/lib/jvm/java-14-openjdk-amd64
   > > > Task :errorProneSkipped
   > > > WARNING: errorprone disabled (skipped on non-nightly runs)
   > > > Task :lucene:core:compileJava
   > > > warning: using incubating module(s): jdk.incubator.foreign
   > > > /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:27: error: cannot find symbol
   > > > import jdk.incubator.foreign.ResourceScope;
   > > > ^
   > > > symbol:   class ResourceScope
   > > > location: package jdk.incubator.foreign
   > > > /mnt/c/Users/jatin/home-ubuntu/sandboxes/vectorAPI-effort/lucene-jdk-foreign/lucene/lucene/core/src/java/org/apache/lucene/store/MemorySegmentIndexInput.java:51: error: cannot find symbol
   > > > final ResourceScope scope;
   > > > ^
   > > > symbol:   class ResourceScope
   > > 
   > > 
   > > Your help will be greatly appriciated.
   > > Best Regards
   > 
   > Hi,
   > 
   > the current branch works with the release candidate builds of OpenJDK 17. If you get errors like this, you may possibly use a OpenJDK 17 version before Early Access Build 25.
   > 
   > Ca you post `java -version`and tell us if you maybe compiledyour own version?
   Hi @uschindler, 
   java --version
   openjdk 17-ea 2021-09-14
   OpenJDK Runtime Environment (build 17-ea+22-1964)
   OpenJDK 64-Bit Server VM (build 17-ea+22-1964, mixed mode, sharing)


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