You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2023/03/09 20:27:38 UTC

[lucene] branch main updated: Implement MMapDirectory with Java 20 Project Panama Preview API (#12188)

This is an automated email from the ASF dual-hosted git repository.

uschindler pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new e4d8a5c5cb7 Implement MMapDirectory with Java 20 Project Panama Preview API (#12188)
e4d8a5c5cb7 is described below

commit e4d8a5c5cb799f70f09d627f243422bf86761208
Author: Uwe Schindler <us...@apache.org>
AuthorDate: Thu Mar 9 21:27:31 2023 +0100

    Implement MMapDirectory with Java 20 Project Panama Preview API (#12188)
---
 build.gradle                                       |   1 +
 buildSrc/scriptDepVersions.gradle                  |   1 +
 gradle/generation/panama-foreign.gradle            |  65 +++
 .../panama-foreign/ExtractForeignAPI.java          | 132 +++++
 gradle/java/memorysegment-mrjar.gradle             |  76 +--
 lucene/CHANGES.txt                                 |  12 +
 lucene/core/src/generated/jdk/README.md            |  43 ++
 .../src/generated/jdk/panama-foreign-jdk19.apijar  | Bin 0 -> 20079 bytes
 .../src/generated/jdk/panama-foreign-jdk20.apijar  | Bin 0 -> 18785 bytes
 .../org/apache/lucene/store/MMapDirectory.java     |   6 +-
 .../lucene/store/MemorySegmentIndexInput.java      | 588 +++++++++++++++++++++
 .../store/MemorySegmentIndexInputProvider.java     | 120 +++++
 .../org/apache/lucene/store/TestMmapDirectory.java |   4 +-
 .../lucene/distribution/TestModularLayer.java      |  18 +-
 14 files changed, 1002 insertions(+), 64 deletions(-)

diff --git a/build.gradle b/build.gradle
index 6032f3a72cc..3fe0e8776e6 100644
--- a/build.gradle
+++ b/build.gradle
@@ -158,6 +158,7 @@ apply from: file('gradle/generation/javacc.gradle')
 apply from: file('gradle/generation/forUtil.gradle')
 apply from: file('gradle/generation/antlr.gradle')
 apply from: file('gradle/generation/unicode-test-classes.gradle')
+apply from: file('gradle/generation/panama-foreign.gradle')
 
 apply from: file('gradle/datasets/external-datasets.gradle')
 
diff --git a/buildSrc/scriptDepVersions.gradle b/buildSrc/scriptDepVersions.gradle
index 6f25cdbf4fd..8751da63249 100644
--- a/buildSrc/scriptDepVersions.gradle
+++ b/buildSrc/scriptDepVersions.gradle
@@ -22,6 +22,7 @@
 ext {
   scriptDepVersions = [
       "apache-rat": "0.14",
+      "asm": "9.4",
       "commons-codec": "1.13",
       "ecj": "3.30.0",
       "flexmark": "0.61.24",
diff --git a/gradle/generation/panama-foreign.gradle b/gradle/generation/panama-foreign.gradle
new file mode 100644
index 00000000000..694c4656e2f
--- /dev/null
+++ b/gradle/generation/panama-foreign.gradle
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+
+def resources = scriptResources(buildscript)
+
+configure(project(":lucene:core")) {
+  ext {
+    apijars = file('src/generated/jdk');
+    panamaJavaVersions = [ 19, 20 ]
+  }
+  
+  configurations {
+    apiextractor
+  }
+
+  dependencies {
+    apiextractor "org.ow2.asm:asm:${scriptDepVersions['asm']}"
+  }
+
+  for (jdkVersion : panamaJavaVersions) {
+    def task = tasks.create(name: "generatePanamaForeignApiJar${jdkVersion}", type: JavaExec) {
+      description "Regenerate the API-only JAR file with public Panama Foreign API from JDK ${jdkVersion}"
+      group "generation"
+      
+      javaLauncher = javaToolchains.launcherFor {
+        languageVersion = JavaLanguageVersion.of(jdkVersion)
+      }
+      
+      onlyIf {
+        try {
+          javaLauncher.get()
+          return true
+        } catch (Exception e) {
+          logger.warn('Launcher for Java {} is not available; skipping regeneration of Panama Foreign API JAR.', jdkVersion)
+          logger.warn('Error: {}', e.cause?.message)
+          logger.warn("Please make sure to point env 'JAVA{}_HOME' to exactly JDK version {} or enable Gradle toolchain auto-download.", jdkVersion, jdkVersion)
+          return false
+        }
+      }
+      
+      classpath = configurations.apiextractor
+      mainClass = file("${resources}/ExtractForeignAPI.java") as String
+      args = [
+        jdkVersion,
+        new File(apijars, "panama-foreign-jdk${jdkVersion}.apijar"),
+      ]
+    }
+
+    regenerate.dependsOn task
+  }
+}
diff --git a/gradle/generation/panama-foreign/ExtractForeignAPI.java b/gradle/generation/panama-foreign/ExtractForeignAPI.java
new file mode 100644
index 00000000000..ca20528cac5
--- /dev/null
+++ b/gradle/generation/panama-foreign/ExtractForeignAPI.java
@@ -0,0 +1,132 @@
+/*
+ * 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.
+ */
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.attribute.FileTime;
+import java.time.Instant;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+import org.objectweb.asm.AnnotationVisitor;
+import org.objectweb.asm.ClassReader;
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.FieldVisitor;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.Type;
+
+public final class ExtractForeignAPI {
+  
+  private static final FileTime FIXED_FILEDATE = FileTime.from(Instant.parse("2022-01-01T00:00:00Z"));
+  
+  public static void main(String... args) throws IOException {
+    if (args.length != 2) {
+      throw new IllegalArgumentException("Need two parameters: java version, output file");
+    }
+    if (Integer.parseInt(args[0]) != Runtime.version().feature()) {
+      throw new IllegalStateException("Incorrect java version: " + Runtime.version().feature());
+    }
+    var outputPath = Paths.get(args[1]);
+    var javaBaseModule = Paths.get(URI.create("jrt:/")).resolve("java.base").toRealPath();
+    var fileMatcher = javaBaseModule.getFileSystem().getPathMatcher("glob:java/{lang/foreign/*,nio/channels/FileChannel}.class");
+    try (var out = new ZipOutputStream(Files.newOutputStream(outputPath)); var stream = Files.walk(javaBaseModule)) {
+      var filesToExtract = stream.map(javaBaseModule::relativize).filter(fileMatcher::matches).sorted().collect(Collectors.toList());
+      for (Path relative : filesToExtract) {
+        System.out.println("Processing class file: " + relative);
+        try (var in = Files.newInputStream(javaBaseModule.resolve(relative))) {
+          final var reader = new ClassReader(in);
+          final var cw = new ClassWriter(0);
+          reader.accept(new Cleaner(cw), ClassReader.SKIP_CODE | ClassReader.SKIP_DEBUG | ClassReader.SKIP_FRAMES);
+          out.putNextEntry(new ZipEntry(relative.toString()).setLastModifiedTime(FIXED_FILEDATE));
+          out.write(cw.toByteArray());
+          out.closeEntry();
+        }
+      }
+    }
+  }
+  
+  static class Cleaner extends ClassVisitor {
+    private static final String PREVIEW_ANN = "jdk/internal/javac/PreviewFeature";
+    private static final String PREVIEW_ANN_DESCR = Type.getObjectType(PREVIEW_ANN).getDescriptor();
+    
+    private boolean completelyHidden = false;
+    
+    Cleaner(ClassWriter out) {
+      super(Opcodes.ASM9, out);
+    }
+    
+    private boolean isHidden(int access) {
+      return completelyHidden || (access & (Opcodes.ACC_PROTECTED | Opcodes.ACC_PUBLIC)) == 0;
+    }
+
+    @Override
+    public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) {
+      super.visit(Opcodes.V11, access, name, signature, superName, interfaces);
+      completelyHidden = isHidden(access);
+    }
+
+    @Override
+    public AnnotationVisitor visitAnnotation(String descriptor, boolean visible) {
+      return Objects.equals(descriptor, PREVIEW_ANN_DESCR) ? null : super.visitAnnotation(descriptor, visible);
+    }
+
+    @Override
+    public FieldVisitor visitField(int access, String name, String descriptor, String signature, Object value) {
+      if (isHidden(access)) {
+        return null;
+      }
+      return new FieldVisitor(Opcodes.ASM9, super.visitField(access, name, descriptor, signature, value)) {
+        @Override
+        public AnnotationVisitor visitAnnotation(String descriptor, boolean visible) {
+          return Objects.equals(descriptor, PREVIEW_ANN_DESCR) ? null : super.visitAnnotation(descriptor, visible);
+        }
+      };
+    }
+
+    @Override
+    public MethodVisitor visitMethod(int access, String name, String descriptor, String signature, String[] exceptions) {
+      if (isHidden(access)) {
+        return null;
+      }
+      return new MethodVisitor(Opcodes.ASM9, super.visitMethod(access, name, descriptor, signature, exceptions)) {
+        @Override
+        public AnnotationVisitor visitAnnotation(String descriptor, boolean visible) {
+          return Objects.equals(descriptor, PREVIEW_ANN_DESCR) ? null : super.visitAnnotation(descriptor, visible);
+        }
+      };
+    }
+
+    @Override
+    public void visitInnerClass(String name, String outerName, String innerName, int access) {
+      if (!Objects.equals(outerName, PREVIEW_ANN)) {
+        super.visitInnerClass(name, outerName, innerName, access);
+      }
+    }
+    
+    @Override
+    public void visitPermittedSubclass​(String c) {
+    }
+
+  }
+  
+}
diff --git a/gradle/java/memorysegment-mrjar.gradle b/gradle/java/memorysegment-mrjar.gradle
index f36f8873b39..137f8a3c567 100644
--- a/gradle/java/memorysegment-mrjar.gradle
+++ b/gradle/java/memorysegment-mrjar.gradle
@@ -15,70 +15,40 @@
  * limitations under the License.
  */
 
-// Produce an MR-JAR with Java 19 MemorySegment implementation for MMapDirectory
+// Produce an MR-JAR with Java 19+ MemorySegment implementation for MMapDirectory
 
 configure(project(":lucene:core")) {
   plugins.withType(JavaPlugin) {
-    sourceSets {
-      main19 {
+    for (jdkVersion : panamaJavaVersions) {
+      sourceSets.create("main${jdkVersion}") {
         java {
-          srcDirs = ['src/java19']
+          srcDirs = ["src/java${jdkVersion}"]
         }
       }
-    }
-
-    configurations {
-      // Inherit any dependencies from the main source set.
-      main19Implementation.extendsFrom implementation
-    }
-    
-    dependencies {
-      // We need the main classes to compile our Java 19 pieces.
-      main19Implementation sourceSets.main.output
-    }
+      configurations["main${jdkVersion}Implementation"].extendsFrom(configurations['implementation'])
+      dependencies.add("main${jdkVersion}Implementation", sourceSets.main.output)
 
-    def patchClassFiles = { DirectoryProperty destinationDirectory, int expectedMajor ->
-      destinationDirectory.getAsFileTree().matching(new PatternSet().include('**/*.class')).visit{ details ->
-        if (!details.directory) {
-          logger.info("Patching: ${details.file}")
-          new RandomAccessFile(details.file, 'rw').withCloseable { f ->
-            int magic = f.readInt();
-            if (magic != (int)0xCAFEBABE) {
-              throw new GradleException("Invalid Java class file magic ${String.format("0x%08X", magic)}: ${details.file}")
-            }
-            f.seek(6L)
-            short major = f.readShort()
-            if (major != expectedMajor) {
-              throw new GradleException("Invalid Java class file version ${major}: ${details.file}")
-            }
-            // patch the minor version to 0 (remove preview flag):
-            f.seek(4L)
-            f.writeShort(0)
-          }
-        }
-      }
-    }
-
-    tasks.named('compileMain19Java').configure {
-      javaCompiler = javaToolchains.compilerFor {
-        languageVersion = JavaLanguageVersion.of(19)
-      }
-      
-      // undo alternative JDK support:
-      options.forkOptions.javaHome = null
-      
-      sourceCompatibility = 19
-      targetCompatibility = 19
-      options.compilerArgs += ["--release", 19 as String, "--enable-preview"]
-      
-      doLast {
-        patchClassFiles(destinationDirectory, 63)
+      tasks.named("compileMain${jdkVersion}Java").configure {
+        def apijar = new File(apijars, "panama-foreign-jdk${jdkVersion}.apijar")
+        
+        inputs.file(apijar)
+        
+        int releaseIndex = options.compilerArgs.indexOf("--release")
+        options.compilerArgs.removeAt(releaseIndex)
+        options.compilerArgs.removeAt(releaseIndex)
+        options.compilerArgs += [
+          "-Xlint:-options",
+          "--patch-module", "java.base=${apijar}",
+          "--add-exports", "java.base/java.lang.foreign=ALL-UNNAMED",
+        ]
       }
     }
     
     tasks.named('jar').configure {
-      into('META-INF/versions/19') {
-        from sourceSets.main19.output
+      for (jdkVersion : panamaJavaVersions) {
+        into("META-INF/versions/${jdkVersion}") {
+          from sourceSets["main${jdkVersion}"].output
+        }
       }
 
       manifest.attributes(
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 0a0a265c8b0..56594385c75 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -125,6 +125,12 @@ New Features
 * GITHUB#12054: Introduce a new KeywordField for simple and efficient
   filtering, sorting and faceting. (Adrien Grand)
 
+* GITHUB#12188: Add support for Java 20 foreign memory API. If exactly Java 19
+  or 20 is used, MMapDirectory will mmap Lucene indexes in chunks of 16 GiB
+  (instead of 1 GiB) and indexes closed while queries are running can no longer
+  crash the JVM. To disable this feature, pass the following sysprop on Java command line:
+  "-Dorg.apache.lucene.store.MMapDirectory.enableMemorySegments=false" (Uwe Schindler)
+
 Improvements
 ---------------------
 
@@ -169,6 +175,12 @@ Build
 
 * GITHUB#12131: Generate gradle.properties from gradlew, if absent (Colvin Cowie, Uwe Schindler)
 
+* GITHUB#12188: Building the lucene-core MR-JAR file is now possible without installing
+  additionally required Java versions (Java 19, Java 20,...). For compilation, a special
+  JAR file with Panama-foreign API signatures of each supported Java version was added to
+  source tree. Those can be regenerated an demand with "gradlew :lucene:core:regenerate".
+  (Uwe Schindler)
+
 Other
 ---------------------
 
diff --git a/lucene/core/src/generated/jdk/README.md b/lucene/core/src/generated/jdk/README.md
new file mode 100644
index 00000000000..371bbebf851
--- /dev/null
+++ b/lucene/core/src/generated/jdk/README.md
@@ -0,0 +1,43 @@
+<!--
+    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.
+ -->
+
+# Generated Java API signatures files
+
+This directory contains generated `.apijar` files. Those are special JAR files containing
+class files that only have public signatures of certain packages of the Java class
+library, but no bytecode at all. Those files are only used to compile the MR-JAR of Apache
+Lucene while allowing to link against APIs only provided as preview APIs in future
+JDK versions.
+
+`.apijar` files are provided for developer's convenience in the Lucene source tree.
+They are not part of Lucene's APIs or source code and are not part of binary releases.
+See them as binary blobs with encoded information also provided through the public
+[Javadocs](https://docs.oracle.com/en/java/javase/) of the corresponding Java
+class library. They contain **no** program code.
+
+This allows Lucene developers to compile the code without downloading a copy of all
+supported JDK versions (Java 19, Java 20,...).
+
+To regenerate those files call `gradlew :lucene:core:regenerate`. While doing this
+you need to either have
+[Gradle toolchain auto-provisioning](https://docs.gradle.org/current/userguide/toolchains.html#sec:provisioning)
+enabled (this is the default for Lucene) or use environment variables like `JAVA19_HOME`
+to point the Lucene build system to missing JDK versions. The regeneration task prints
+a warning if a specific JDK is missing, leaving the already existing `.apijar` file
+untouched.
+
+The extraction is done with the ASM library, see `ExtractForeignAPI.java` source code.
diff --git a/lucene/core/src/generated/jdk/panama-foreign-jdk19.apijar b/lucene/core/src/generated/jdk/panama-foreign-jdk19.apijar
new file mode 100644
index 00000000000..3c087ebbf97
Binary files /dev/null and b/lucene/core/src/generated/jdk/panama-foreign-jdk19.apijar differ
diff --git a/lucene/core/src/generated/jdk/panama-foreign-jdk20.apijar b/lucene/core/src/generated/jdk/panama-foreign-jdk20.apijar
new file mode 100644
index 00000000000..b2ce3f750a9
Binary files /dev/null and b/lucene/core/src/generated/jdk/panama-foreign-jdk20.apijar differ
diff --git a/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java b/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
index d5bb94d9e92..58be07ab437 100644
--- a/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
@@ -346,7 +346,7 @@ public class MMapDirectory extends FSDirectory {
     }
     final var lookup = MethodHandles.lookup();
     final int runtimeVersion = Runtime.version().feature();
-    if (runtimeVersion == 19) {
+    if (runtimeVersion == 19 || runtimeVersion == 20) {
       try {
         final var cls = lookup.findClass("org.apache.lucene.store.MemorySegmentIndexInputProvider");
         // we use method handles, so we do not need to deal with setAccessible as we have private
@@ -366,9 +366,9 @@ public class MMapDirectory extends FSDirectory {
         throw new LinkageError(
             "MemorySegmentIndexInputProvider is missing in Lucene JAR file", cnfe);
       }
-    } else if (runtimeVersion >= 20) {
+    } else if (runtimeVersion >= 21) {
       LOG.warning(
-          "You are running with Java 20 or later. To make full use of MMapDirectory, please update Apache Lucene.");
+          "You are running with Java 21 or later. To make full use of MMapDirectory, please update Apache Lucene.");
     }
     return new MappedByteBufferIndexInputProvider();
   }
diff --git a/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInput.java b/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInput.java
new file mode 100644
index 00000000000..7b2216add78
--- /dev/null
+++ b/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInput.java
@@ -0,0 +1,588 @@
+/*
+ * 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.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.lang.foreign.ValueLayout;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Objects;
+import org.apache.lucene.util.ArrayUtil;
+
+/**
+ * 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>).
+ */
+@SuppressWarnings("preview")
+abstract class MemorySegmentIndexInput extends IndexInput implements RandomAccessInput {
+  static final ValueLayout.OfByte LAYOUT_BYTE = ValueLayout.JAVA_BYTE;
+  static final ValueLayout.OfShort LAYOUT_LE_SHORT =
+      ValueLayout.JAVA_SHORT_UNALIGNED.withOrder(ByteOrder.LITTLE_ENDIAN);
+  static final ValueLayout.OfInt LAYOUT_LE_INT =
+      ValueLayout.JAVA_INT_UNALIGNED.withOrder(ByteOrder.LITTLE_ENDIAN);
+  static final ValueLayout.OfLong LAYOUT_LE_LONG =
+      ValueLayout.JAVA_LONG_UNALIGNED.withOrder(ByteOrder.LITTLE_ENDIAN);
+  static final ValueLayout.OfFloat LAYOUT_LE_FLOAT =
+      ValueLayout.JAVA_FLOAT_UNALIGNED.withOrder(ByteOrder.LITTLE_ENDIAN);
+
+  final long length;
+  final long chunkSizeMask;
+  final int chunkSizePower;
+  final Arena arena;
+  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,
+      Arena arena,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(arena.scope()::equals);
+    if (segments.length == 1) {
+      return new SingleSegmentImpl(resourceDescription, arena, segments[0], length, chunkSizePower);
+    } else {
+      return new MultiSegmentImpl(resourceDescription, arena, segments, 0, length, chunkSizePower);
+    }
+  }
+
+  private MemorySegmentIndexInput(
+      String resourceDescription,
+      Arena arena,
+      MemorySegment[] segments,
+      long length,
+      int chunkSizePower) {
+    super(resourceDescription);
+    this.arena = arena;
+    this.segments = segments;
+    this.length = length;
+    this.chunkSizePower = chunkSizePower;
+    this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.curSegment = segments[0];
+  }
+
+  void ensureOpen() {
+    if (curSegment == null) {
+      throw alreadyClosed(null);
+    }
+  }
+
+  // the unused parameter is just to silence javac about unused variables
+  RuntimeException handlePositionalIOOBE(RuntimeException unused, String action, long pos)
+      throws IOException {
+    if (pos < 0L) {
+      return new IllegalArgumentException(action + " negative position (pos=" + pos + "): " + this);
+    } else {
+      throw new EOFException(action + " past EOF (pos=" + pos + "): " + this);
+    }
+  }
+
+  // the unused parameter is just to silence javac about unused variables
+  AlreadyClosedException alreadyClosed(RuntimeException unused) {
+    return new AlreadyClosedException("Already closed: " + this);
+  }
+
+  @Override
+  public final byte readByte() throws IOException {
+    try {
+      final byte v = curSegment.get(LAYOUT_BYTE, 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 = curSegment.get(LAYOUT_BYTE, curPosition);
+      curPosition++;
+      return v;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public final void readBytes(byte[] b, int offset, int len) throws IOException {
+    try {
+      MemorySegment.copy(curSegment, LAYOUT_BYTE, curPosition, b, offset, len);
+      curPosition += len;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      readBytesBoundary(b, offset, len);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  private void readBytesBoundary(byte[] b, int offset, int len) throws IOException {
+    try {
+      long curAvail = curSegment.byteSize() - curPosition;
+      while (len > curAvail) {
+        MemorySegment.copy(curSegment, LAYOUT_BYTE, curPosition, b, offset, (int) 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();
+      }
+      MemorySegment.copy(curSegment, LAYOUT_BYTE, curPosition, b, offset, len);
+      curPosition += len;
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public void readInts(int[] dst, int offset, int length) throws IOException {
+    try {
+      MemorySegment.copy(curSegment, LAYOUT_LE_INT, curPosition, dst, offset, length);
+      curPosition += Integer.BYTES * (long) length;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException iobe) {
+      super.readInts(dst, offset, length);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public void readLongs(long[] dst, int offset, int length) throws IOException {
+    try {
+      MemorySegment.copy(curSegment, LAYOUT_LE_LONG, curPosition, dst, offset, length);
+      curPosition += Long.BYTES * (long) length;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException iobe) {
+      super.readLongs(dst, offset, length);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public void readFloats(float[] dst, int offset, int length) throws IOException {
+    try {
+      MemorySegment.copy(curSegment, LAYOUT_LE_FLOAT, curPosition, dst, offset, length);
+      curPosition += Float.BYTES * (long) length;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException iobe) {
+      super.readFloats(dst, offset, length);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public final short readShort() throws IOException {
+    try {
+      final short v = curSegment.get(LAYOUT_LE_SHORT, curPosition);
+      curPosition += Short.BYTES;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      return super.readShort();
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public final int readInt() throws IOException {
+    try {
+      final int v = curSegment.get(LAYOUT_LE_INT, curPosition);
+      curPosition += Integer.BYTES;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      return super.readInt();
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public final long readLong() throws IOException {
+    try {
+      final long v = curSegment.get(LAYOUT_LE_LONG, curPosition);
+      curPosition += Long.BYTES;
+      return v;
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException e) {
+      return super.readLong();
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public long getFilePointer() {
+    ensureOpen();
+    return (((long) curSegmentIndex) << chunkSizePower) + curPosition;
+  }
+
+  @Override
+  public void seek(long pos) throws IOException {
+    ensureOpen();
+    // we use >> here to preserve negative, so we will catch AIOOBE,
+    // in case pos + offset overflows.
+    final int si = (int) (pos >> chunkSizePower);
+    try {
+      if (si != curSegmentIndex) {
+        final MemorySegment seg = segments[si];
+        // write values, on exception all is unchanged
+        this.curSegmentIndex = si;
+        this.curSegment = seg;
+      }
+      this.curPosition = Objects.checkIndex(pos & chunkSizeMask, curSegment.byteSize() + 1);
+    } catch (IndexOutOfBoundsException e) {
+      throw handlePositionalIOOBE(e, "seek", pos);
+    }
+  }
+
+  @Override
+  public byte readByte(long pos) throws IOException {
+    try {
+      final int si = (int) (pos >> chunkSizePower);
+      return segments[si].get(LAYOUT_BYTE, pos & chunkSizeMask);
+    } catch (IndexOutOfBoundsException ioobe) {
+      throw handlePositionalIOOBE(ioobe, "read", pos);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  // used only by random access methods to handle reads across boundaries
+  private void setPos(long pos, int si) throws IOException {
+    try {
+      final MemorySegment seg = segments[si];
+      // write values, on exception above all is unchanged
+      this.curPosition = pos & chunkSizeMask;
+      this.curSegmentIndex = si;
+      this.curSegment = seg;
+    } catch (IndexOutOfBoundsException ioobe) {
+      throw handlePositionalIOOBE(ioobe, "read", pos);
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public short readShort(long pos) throws IOException {
+    final int si = (int) (pos >> chunkSizePower);
+    try {
+      return segments[si].get(LAYOUT_LE_SHORT, pos & chunkSizeMask);
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException ioobe) {
+      // either it's a boundary, or read past EOF, fall back:
+      setPos(pos, si);
+      return readShort();
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public int readInt(long pos) throws IOException {
+    final int si = (int) (pos >> chunkSizePower);
+    try {
+      return segments[si].get(LAYOUT_LE_INT, pos & chunkSizeMask);
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException ioobe) {
+      // either it's a boundary, or read past EOF, fall back:
+      setPos(pos, si);
+      return readInt();
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public long readLong(long pos) throws IOException {
+    final int si = (int) (pos >> chunkSizePower);
+    try {
+      return segments[si].get(LAYOUT_LE_LONG, pos & chunkSizeMask);
+    } catch (
+        @SuppressWarnings("unused")
+        IndexOutOfBoundsException ioobe) {
+      // either it's a boundary, or read past EOF, fall back:
+      setPos(pos, si);
+      return readLong();
+    } catch (NullPointerException | IllegalStateException e) {
+      throw alreadyClosed(e);
+    }
+  }
+
+  @Override
+  public final long length() {
+    return length;
+  }
+
+  @Override
+  public final MemorySegmentIndexInput clone() {
+    final MemorySegmentIndexInput clone = buildSlice((String) null, 0L, this.length);
+    try {
+      clone.seek(getFilePointer());
+    } catch (IOException ioe) {
+      throw new AssertionError(ioe);
+    }
+
+    return clone;
+  }
+
+  /**
+   * Creates a slice of this index input, with the given description, offset, and length. The slice
+   * is seeked to the beginning.
+   */
+  @Override
+  public final MemorySegmentIndexInput slice(String sliceDescription, long offset, long length) {
+    if (offset < 0 || length < 0 || offset + length > this.length) {
+      throw new IllegalArgumentException(
+          "slice() "
+              + sliceDescription
+              + " out of bounds: offset="
+              + offset
+              + ",length="
+              + length
+              + ",fileLength="
+              + this.length
+              + ": "
+              + this);
+    }
+
+    return buildSlice(sliceDescription, offset, length);
+  }
+
+  /** Builds the actual sliced IndexInput (may apply extra offset in subclasses). * */
+  MemorySegmentIndexInput buildSlice(String sliceDescription, long offset, long length) {
+    ensureOpen();
+
+    final long sliceEnd = offset + length;
+    final int startIndex = (int) (offset >>> chunkSizePower);
+    final int endIndex = (int) (sliceEnd >>> chunkSizePower);
+
+    // we always allocate one more slice, the last one may be a 0 byte one after truncating with
+    // asSlice():
+    final MemorySegment slices[] = ArrayUtil.copyOfSubArray(segments, startIndex, endIndex + 1);
+
+    // set the last segment's limit for the sliced view.
+    slices[slices.length - 1] = slices[slices.length - 1].asSlice(0L, sliceEnd & chunkSizeMask);
+
+    offset = offset & chunkSizeMask;
+
+    final String newResourceDescription = getFullSliceDescription(sliceDescription);
+    if (slices.length == 1) {
+      return new SingleSegmentImpl(
+          newResourceDescription,
+          null, // clones don't have an Arena, as they can't close)
+          slices[0].asSlice(offset, length),
+          length,
+          chunkSizePower);
+    } else {
+      return new MultiSegmentImpl(
+          newResourceDescription,
+          null, // clones don't have an Arena, as they can't close)
+          slices,
+          offset,
+          length,
+          chunkSizePower);
+    }
+  }
+
+  @Override
+  public final void close() throws IOException {
+    if (curSegment == null) {
+      return;
+    }
+
+    // make sure all accesses to this IndexInput instance throw NPE:
+    curSegment = null;
+    Arrays.fill(segments, null);
+
+    // the master IndexInput has an Arena and is able
+    // to release all resources (unmap segments) - a
+    // side effect is that other threads still using clones
+    // will throw IllegalStateException
+    if (arena != null) {
+      arena.close();
+    }
+  }
+
+  /** Optimization of MemorySegmentIndexInput for when there is only one segment. */
+  static final class SingleSegmentImpl extends MemorySegmentIndexInput {
+
+    SingleSegmentImpl(
+        String resourceDescription,
+        Arena arena,
+        MemorySegment segment,
+        long length,
+        int chunkSizePower) {
+      super(resourceDescription, arena, new MemorySegment[] {segment}, length, chunkSizePower);
+      this.curSegmentIndex = 0;
+    }
+
+    @Override
+    public void seek(long pos) throws IOException {
+      ensureOpen();
+      try {
+        curPosition = Objects.checkIndex(pos, length + 1);
+      } catch (IndexOutOfBoundsException e) {
+        throw handlePositionalIOOBE(e, "seek", pos);
+      }
+    }
+
+    @Override
+    public long getFilePointer() {
+      ensureOpen();
+      return curPosition;
+    }
+
+    @Override
+    public byte readByte(long pos) throws IOException {
+      try {
+        return curSegment.get(LAYOUT_BYTE, pos);
+      } catch (IndexOutOfBoundsException e) {
+        throw handlePositionalIOOBE(e, "read", pos);
+      } catch (NullPointerException | IllegalStateException e) {
+        throw alreadyClosed(e);
+      }
+    }
+
+    @Override
+    public short readShort(long pos) throws IOException {
+      try {
+        return curSegment.get(LAYOUT_LE_SHORT, pos);
+      } catch (IndexOutOfBoundsException e) {
+        throw handlePositionalIOOBE(e, "read", pos);
+      } catch (NullPointerException | IllegalStateException e) {
+        throw alreadyClosed(e);
+      }
+    }
+
+    @Override
+    public int readInt(long pos) throws IOException {
+      try {
+        return curSegment.get(LAYOUT_LE_INT, pos);
+      } catch (IndexOutOfBoundsException e) {
+        throw handlePositionalIOOBE(e, "read", pos);
+      } catch (NullPointerException | IllegalStateException e) {
+        throw alreadyClosed(e);
+      }
+    }
+
+    @Override
+    public long readLong(long pos) throws IOException {
+      try {
+        return curSegment.get(LAYOUT_LE_LONG, pos);
+      } catch (IndexOutOfBoundsException e) {
+        throw handlePositionalIOOBE(e, "read", pos);
+      } catch (NullPointerException | IllegalStateException e) {
+        throw alreadyClosed(e);
+      }
+    }
+  }
+
+  /** This class adds offset support to MemorySegmentIndexInput, which is needed for slices. */
+  static final class MultiSegmentImpl extends MemorySegmentIndexInput {
+    private final long offset;
+
+    MultiSegmentImpl(
+        String resourceDescription,
+        Arena arena,
+        MemorySegment[] segments,
+        long offset,
+        long length,
+        int chunkSizePower) {
+      super(resourceDescription, arena, segments, length, chunkSizePower);
+      this.offset = offset;
+      try {
+        seek(0L);
+      } catch (IOException ioe) {
+        throw new AssertionError(ioe);
+      }
+      assert curSegment != null && curSegmentIndex >= 0;
+    }
+
+    @Override
+    RuntimeException handlePositionalIOOBE(RuntimeException unused, String action, long pos)
+        throws IOException {
+      return super.handlePositionalIOOBE(unused, action, pos - offset);
+    }
+
+    @Override
+    public void seek(long pos) throws IOException {
+      assert pos >= 0L : "negative position";
+      super.seek(pos + offset);
+    }
+
+    @Override
+    public long getFilePointer() {
+      return super.getFilePointer() - offset;
+    }
+
+    @Override
+    public byte readByte(long pos) throws IOException {
+      return super.readByte(pos + offset);
+    }
+
+    @Override
+    public short readShort(long pos) throws IOException {
+      return super.readShort(pos + offset);
+    }
+
+    @Override
+    public int readInt(long pos) throws IOException {
+      return super.readInt(pos + offset);
+    }
+
+    @Override
+    public long readLong(long pos) throws IOException {
+      return super.readLong(pos + offset);
+    }
+
+    @Override
+    MemorySegmentIndexInput buildSlice(String sliceDescription, long ofs, long length) {
+      return super.buildSlice(sliceDescription, this.offset + ofs, length);
+    }
+  }
+}
diff --git a/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInputProvider.java b/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInputProvider.java
new file mode 100644
index 00000000000..5fc2533af28
--- /dev/null
+++ b/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInputProvider.java
@@ -0,0 +1,120 @@
+/*
+ * 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.IOException;
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.logging.Logger;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.Unwrappable;
+
+@SuppressWarnings("preview")
+final class MemorySegmentIndexInputProvider implements MMapDirectory.MMapIndexInputProvider {
+
+  public MemorySegmentIndexInputProvider() {
+    var log = Logger.getLogger(getClass().getName());
+    log.info(
+        "Using MemorySegmentIndexInput with Java 20; to disable start with -D"
+            + MMapDirectory.ENABLE_MEMORY_SEGMENTS_SYSPROP
+            + "=false");
+  }
+
+  @Override
+  public IndexInput openInput(Path path, IOContext context, int chunkSizePower, boolean preload)
+      throws IOException {
+    final String resourceDescription = "MemorySegmentIndexInput(path=\"" + path.toString() + "\")";
+
+    // Work around for JDK-8259028: we need to unwrap our test-only file system layers
+    path = Unwrappable.unwrapAll(path);
+
+    boolean success = false;
+    final Arena arena = Arena.openShared();
+    try (var fc = FileChannel.open(path, StandardOpenOption.READ)) {
+      final long fileSize = fc.size();
+      final IndexInput in =
+          MemorySegmentIndexInput.newInstance(
+              resourceDescription,
+              arena,
+              map(arena, resourceDescription, fc, chunkSizePower, preload, fileSize),
+              fileSize,
+              chunkSizePower);
+      success = true;
+      return in;
+    } finally {
+      if (success == false) {
+        arena.close();
+      }
+    }
+  }
+
+  @Override
+  public long getDefaultMaxChunkSize() {
+    return Constants.JRE_IS_64BIT ? (1L << 34) : (1L << 28);
+  }
+
+  @Override
+  public boolean isUnmapSupported() {
+    return true;
+  }
+
+  @Override
+  public String getUnmapNotSupportedReason() {
+    return null;
+  }
+
+  private final MemorySegment[] map(
+      Arena arena,
+      String resourceDescription,
+      FileChannel fc,
+      int chunkSizePower,
+      boolean preload,
+      long length)
+      throws IOException {
+    if ((length >>> chunkSizePower) >= Integer.MAX_VALUE)
+      throw new IllegalArgumentException("File too big for chunk size: " + resourceDescription);
+
+    final long chunkSize = 1L << chunkSizePower;
+
+    // we always allocate one more segments, the last one may be a 0 byte one
+    final int nrSegments = (int) (length >>> chunkSizePower) + 1;
+
+    final MemorySegment[] segments = new MemorySegment[nrSegments];
+
+    long startOffset = 0L;
+    for (int segNr = 0; segNr < nrSegments; segNr++) {
+      final long segSize =
+          (length > (startOffset + chunkSize)) ? chunkSize : (length - startOffset);
+      final MemorySegment segment;
+      try {
+        segment = fc.map(MapMode.READ_ONLY, startOffset, segSize, arena.scope());
+      } catch (IOException ioe) {
+        throw convertMapFailedIOException(ioe, resourceDescription, segSize);
+      }
+      if (preload) {
+        segment.load();
+      }
+      segments[segNr] = segment;
+      startOffset += segSize;
+    }
+    return segments;
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestMmapDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestMmapDirectory.java
index 5faf12484cc..dc638f6a529 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestMmapDirectory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestMmapDirectory.java
@@ -48,9 +48,9 @@ public class TestMmapDirectory extends BaseDirectoryTestCase {
 
   public void testCorrectImplementation() {
     final int runtimeVersion = Runtime.version().feature();
-    if (runtimeVersion == 19) {
+    if (runtimeVersion == 19 || runtimeVersion == 20) {
       assertTrue(
-          "on Java 19 we should use MemorySegmentIndexInputProvider to create mmap IndexInputs",
+          "on Java 19 and Java 20 we should use MemorySegmentIndexInputProvider to create mmap IndexInputs",
           isMemorySegmentImpl());
     } else {
       assertSame(MappedByteBufferIndexInputProvider.class, MMapDirectory.PROVIDER.getClass());
diff --git a/lucene/distribution.tests/src/test/org/apache/lucene/distribution/TestModularLayer.java b/lucene/distribution.tests/src/test/org/apache/lucene/distribution/TestModularLayer.java
index 88ce76f69a7..b3e55f27770 100644
--- a/lucene/distribution.tests/src/test/org/apache/lucene/distribution/TestModularLayer.java
+++ b/lucene/distribution.tests/src/test/org/apache/lucene/distribution/TestModularLayer.java
@@ -183,7 +183,7 @@ public class TestModularLayer extends AbstractLuceneDistributionTest {
             });
   }
 
-  /** Checks that Lucene Core is a MR-JAR and has JDK 19 classes */
+  /** Checks that Lucene Core is a MR-JAR and has Panama foreign classes */
   @Test
   public void testMultiReleaseJar() {
     ModuleLayer bootLayer = ModuleLayer.boot();
@@ -206,12 +206,18 @@ public class TestModularLayer extends AbstractLuceneDistributionTest {
 
               ClassLoader loader = layer.findLoader(coreModuleId);
 
-              Assertions.assertThat(
-                      loader.getResource(
-                          "META-INF/versions/19/org/apache/lucene/store/MemorySegmentIndexInput.class"))
-                  .isNotNull();
+              final Set<Integer> jarVersions = Set.of(19, 20);
+              for (var v : jarVersions) {
+                Assertions.assertThat(
+                        loader.getResource(
+                            "META-INF/versions/"
+                                + v
+                                + "/org/apache/lucene/store/MemorySegmentIndexInput.class"))
+                    .isNotNull();
+              }
 
-              if (Runtime.version().feature() == 19) {
+              final int runtimeVersion = Runtime.version().feature();
+              if (jarVersions.contains(Integer.valueOf(runtimeVersion))) {
                 Assertions.assertThat(
                         loader.loadClass("org.apache.lucene.store.MemorySegmentIndexInput"))
                     .isNotNull();