You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by to...@apache.org on 2022/01/19 15:09:20 UTC

[lucene] branch main updated: LUCENE-8930: script testing in the distribution (#550)

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

tomoko 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 72ba7ae  LUCENE-8930: script testing in the distribution (#550)
72ba7ae is described below

commit 72ba7ae2eed4e69ed5015b51ea9d69009bc82bd4
Author: Dawid Weiss <da...@carrotsearch.com>
AuthorDate: Wed Jan 19 16:09:15 2022 +0100

    LUCENE-8930: script testing in the distribution (#550)
---
 lucene/distribution.tests/build.gradle             |   1 +
 .../AbstractLuceneDistributionTest.java            |  84 ++++++++++++
 .../test/org/apache/lucene/distribution/Sync.java  |  94 ++++++++++++++
 .../lucene/distribution/TestModularLayer.java      |  20 +--
 .../apache/lucene/distribution/TestScripts.java    | 143 +++++++++++++++++++++
 .../distribution/src/binary-release/bin/luke.cmd   |  12 +-
 lucene/distribution/src/binary-release/bin/luke.sh |   3 +-
 lucene/licenses/procfork-1.0.6.jar.sha1            |   1 +
 lucene/licenses/procfork-LICENSE-ASL.txt           |  34 +++++
 lucene/licenses/procfork-NOTICE.txt                |   0
 .../apache/lucene/luke/app/desktop/LukeMain.java   |  42 ++++--
 versions.lock                                      |   1 +
 versions.props                                     |   1 +
 13 files changed, 402 insertions(+), 34 deletions(-)

diff --git a/lucene/distribution.tests/build.gradle b/lucene/distribution.tests/build.gradle
index 30c4ae1..23e4352 100644
--- a/lucene/distribution.tests/build.gradle
+++ b/lucene/distribution.tests/build.gradle
@@ -29,6 +29,7 @@ configurations {
 dependencies {
   binaryDistribution project(path: ":lucene:distribution", configuration: "binaryDirForTests")
 
+  moduleTestImplementation "com.carrotsearch:procfork"
   moduleTestImplementation("com.carrotsearch.randomizedtesting:randomizedtesting-runner", {
     exclude group: "junit"
   })
diff --git a/lucene/distribution.tests/src/test/org/apache/lucene/distribution/AbstractLuceneDistributionTest.java b/lucene/distribution.tests/src/test/org/apache/lucene/distribution/AbstractLuceneDistributionTest.java
new file mode 100644
index 0000000..0a1868f
--- /dev/null
+++ b/lucene/distribution.tests/src/test/org/apache/lucene/distribution/AbstractLuceneDistributionTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.distribution;
+
+import com.carrotsearch.randomizedtesting.RandomizedTest;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Objects;
+import org.assertj.core.api.Assertions;
+import org.junit.BeforeClass;
+
+/**
+ * A parent scaffolding for tests that take a Lucene distribution as input. The location of the
+ * distribution is pointed to by a system property {@link #DISTRIBUTION_PROPERTY}, which by default
+ * is prepared and passed by the gradle build. It can be passed manually if you're testing from the
+ * IDE, for example.
+ *
+ * <p>We do <em>not</em> want any distribution tests to depend on any Lucene classes (including the
+ * test framework) so that there is no risk of accidental classpath space pollution. This also means
+ * the default {@code LuceneTestCase} configuration setup is not used (you have to annotate test for
+ * JUnit, for example).
+ */
+@ThreadLeakScope(ThreadLeakScope.Scope.NONE)
+public abstract class AbstractLuceneDistributionTest extends RandomizedTest {
+  /** A path to a directory with an expanded Lucene distribution. */
+  public static final String DISTRIBUTION_PROPERTY = "lucene.distribution.dir";
+
+  /** The expected distribution version of Lucene modules. */
+  public static final String VERSION_PROPERTY = "lucene.distribution.version";
+
+  /** Resolved and validated {@link #DISTRIBUTION_PROPERTY}. */
+  private static Path distributionPath;
+
+  /** Ensure Lucene classes are not directly visible. */
+  @BeforeClass
+  public static void checkLuceneNotInClasspath() {
+    Assertions.assertThatThrownBy(
+            () -> {
+              Class.forName("org.apache.lucene.index.IndexWriter");
+            })
+        .isInstanceOf(ClassNotFoundException.class);
+  }
+
+  /** Verify the distribution property is provided and points at a valid location. */
+  @BeforeClass
+  public static void parseExternalProperties() {
+    String distributionPropertyValue = System.getProperty(DISTRIBUTION_PROPERTY);
+    if (distributionPropertyValue == null) {
+      throw new AssertionError(DISTRIBUTION_PROPERTY + " property is required for this test.");
+    }
+
+    distributionPath = Paths.get(distributionPropertyValue);
+
+    // Ensure the distribution path is sort of valid.
+    Path topLevelReadme = distributionPath.resolve("README.md");
+    if (!Files.isRegularFile(topLevelReadme)) {
+      throw new AssertionError(
+          DISTRIBUTION_PROPERTY
+              + " property does not seem to point to a top-level distribution directory"
+              + " where this file is present: "
+              + topLevelReadme.toAbsolutePath());
+    }
+  }
+
+  protected static Path getDistributionPath() {
+    return Objects.requireNonNull(distributionPath, "Distribution path not set?");
+  }
+}
diff --git a/lucene/distribution.tests/src/test/org/apache/lucene/distribution/Sync.java b/lucene/distribution.tests/src/test/org/apache/lucene/distribution/Sync.java
new file mode 100644
index 0000000..e37f422
--- /dev/null
+++ b/lucene/distribution.tests/src/test/org/apache/lucene/distribution/Sync.java
@@ -0,0 +1,94 @@
+/*
+ * 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.distribution;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.*;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.*;
+import java.util.stream.Collectors;
+
+final class Sync {
+  private static class Entry {
+    String name;
+    Path path;
+
+    public Entry(Path path) {
+      this.path = path;
+      this.name = path.getFileName().toString();
+    }
+  }
+
+  public void sync(Path source, Path target) throws IOException {
+    List<Entry> sourceEntries = files(source);
+    List<Entry> targetEntries = files(target);
+
+    for (Entry src : sourceEntries) {
+      Path dst = target.resolve(src.name);
+      if (Files.isDirectory(src.path)) {
+        Files.createDirectories(dst);
+        sync(src.path, dst);
+      } else {
+        if (!Files.exists(dst)
+            || Files.size(dst) != Files.size(src.path)
+            || Files.getLastModifiedTime(dst).compareTo(Files.getLastModifiedTime(src.path)) != 0) {
+          Files.copy(
+              src.path,
+              dst,
+              StandardCopyOption.COPY_ATTRIBUTES,
+              StandardCopyOption.REPLACE_EXISTING);
+        }
+      }
+    }
+
+    Set<String> atSource = sourceEntries.stream().map(e -> e.name).collect(Collectors.toSet());
+    targetEntries.stream().filter(v -> !atSource.contains(v.name)).forEach(e -> remove(e.path));
+  }
+
+  private List<Entry> files(Path source) throws IOException {
+    ArrayList<Entry> entries = new ArrayList<>();
+    try (DirectoryStream<Path> ds = Files.newDirectoryStream(source)) {
+      ds.forEach(p -> entries.add(new Entry(p)));
+    }
+    return entries;
+  }
+
+  private static void remove(Path p) {
+    try {
+      Files.walkFileTree(
+          p,
+          new SimpleFileVisitor<>() {
+            @Override
+            public FileVisitResult postVisitDirectory(Path dir, IOException exc)
+                throws IOException {
+              Files.delete(dir);
+              return FileVisitResult.CONTINUE;
+            }
+
+            @Override
+            public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
+                throws IOException {
+              Files.delete(file);
+              return FileVisitResult.CONTINUE;
+            }
+          });
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+}
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 ed9b188..72be33d 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
@@ -51,29 +51,13 @@ import org.junit.Test;
  * default {@code LuceneTestCase} configuration setup is not used (you have to annotate test for
  * JUnit, for example).
  */
-public class TestModularLayer {
-  /** A path to a directory with an expanded Lucene distribution. */
-  private static final String DISTRIBUTION_PROPERTY = "lucene.distribution.dir";
-
-  /** The expected distribution version of Lucene modules. */
-  private static final String VERSION_PROPERTY = "lucene.distribution.version";
-
+public class TestModularLayer extends AbstractLuceneDistributionTest {
   /** Only core Lucene modules, no third party modules. */
   private static Set<ModuleReference> allCoreModules;
 
   /** {@link ModuleFinder} resolving only the Lucene modules. */
   private static ModuleFinder coreModulesFinder;
 
-  /** Ensure Lucene classes are not directly visible. */
-  @BeforeClass
-  public static void checkLuceneNotInClasspath() {
-    Assertions.assertThatThrownBy(
-            () -> {
-              Class.forName("org.apache.lucene.index.IndexWriter");
-            })
-        .isInstanceOf(ClassNotFoundException.class);
-  }
-
   /**
    * We accept external properties that point to the assembled set of distribution modules and to
    * their expected version. These properties are collected and passed by gradle but can be provided
@@ -86,7 +70,7 @@ public class TestModularLayer {
       throw new AssertionError(DISTRIBUTION_PROPERTY + " property is required for this test.");
     }
 
-    Path modulesPath = Paths.get(modulesPropertyValue).resolve("modules");
+    Path modulesPath = getDistributionPath().resolve("modules");
     if (!Files.isDirectory(modulesPath)) {
       throw new AssertionError(
           DISTRIBUTION_PROPERTY
diff --git a/lucene/distribution.tests/src/test/org/apache/lucene/distribution/TestScripts.java b/lucene/distribution.tests/src/test/org/apache/lucene/distribution/TestScripts.java
new file mode 100644
index 0000000..bf7b3f7
--- /dev/null
+++ b/lucene/distribution.tests/src/test/org/apache/lucene/distribution/TestScripts.java
@@ -0,0 +1,143 @@
+/*
+ * 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.distribution;
+
+import com.carrotsearch.procfork.ForkedProcess;
+import com.carrotsearch.procfork.Launcher;
+import com.carrotsearch.procfork.ProcessBuilderLauncher;
+import com.carrotsearch.randomizedtesting.LifecycleScope;
+import com.carrotsearch.randomizedtesting.RandomizedTest;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import org.assertj.core.api.Assertions;
+import org.assertj.core.api.ThrowingConsumer;
+import org.junit.Test;
+
+/** Verify that scripts included in the distribution work. */
+public class TestScripts extends AbstractLuceneDistributionTest {
+  @Test
+  public void testLukeCanBeLaunched() throws Exception {
+    Path distributionPath;
+    if (randomBoolean()) {
+      // Occasionally, be evil: put the distribution in a folder with a space inside. For Uwe.
+      distributionPath = RandomizedTest.newTempDir(LifecycleScope.TEST).resolve("uh oh");
+      Files.createDirectory(distributionPath);
+      new Sync().sync(getDistributionPath(), distributionPath);
+    } else {
+      distributionPath = getDistributionPath();
+    }
+
+    Path lukeScript = resolveScript(distributionPath.resolve("bin").resolve("luke"));
+
+    Launcher launcher =
+        new ProcessBuilderLauncher()
+            .executable(lukeScript)
+            // tweak Windows launcher scripts so that they don't fork asynchronous java.
+            .envvar("DISTRIBUTION_TESTING", "true")
+            .viaShellLauncher()
+            .cwd(distributionPath)
+            .args("--sanity-check");
+
+    execute(
+        launcher,
+        0,
+        5,
+        (output) -> {
+          Assertions.assertThat(output).contains("[Vader] Hello, Luke.");
+        });
+  }
+
+  /** The value of <code>System.getProperty("os.name")</code>. * */
+  public static final String OS_NAME = System.getProperty("os.name");
+  /** True iff running on Windows. */
+  public static final boolean WINDOWS = OS_NAME.startsWith("Windows");
+
+  protected Path resolveScript(Path scriptPath) {
+    List<Path> candidates = new ArrayList<>();
+    candidates.add(scriptPath);
+
+    String fileName = scriptPath.getFileName().toString();
+    if (WINDOWS) {
+      candidates.add(scriptPath.resolveSibling(fileName + ".cmd"));
+      candidates.add(scriptPath.resolveSibling(fileName + ".bat"));
+    } else {
+      candidates.add(scriptPath.resolveSibling(fileName + ".sh"));
+    }
+
+    return candidates.stream()
+        .sequential()
+        .filter(Files::exists)
+        .findFirst()
+        .orElseThrow(() -> new AssertionError("No script found for the base path: " + scriptPath));
+  }
+
+  private static Supplier<Charset> forkedProcessCharset =
+      () -> {
+        // The default charset for a forked java process could be computed for the current
+        // platform but it adds more complexity. For now, assume it's just parseable ascii.
+        return StandardCharsets.US_ASCII;
+      };
+
+  protected String execute(
+      Launcher launcher,
+      int expectedExitCode,
+      long timeoutInSeconds,
+      ThrowingConsumer<String> consumer)
+      throws Exception {
+
+    try (ForkedProcess forkedProcess = launcher.execute()) {
+      String command = forkedProcess.getProcess().info().command().orElse("(unset command name)");
+
+      Charset charset = forkedProcessCharset.get();
+      try {
+        Process p = forkedProcess.getProcess();
+        if (!p.waitFor(timeoutInSeconds, TimeUnit.SECONDS)) {
+          throw new AssertionError("Forked process did not terminate in the expected time");
+        }
+
+        int exitStatus = p.exitValue();
+
+        Assertions.assertThat(exitStatus)
+            .as("forked process exit status")
+            .isEqualTo(expectedExitCode);
+
+        String output = Files.readString(forkedProcess.getProcessOutputFile(), charset);
+        consumer.accept(output);
+        return output;
+      } catch (Throwable t) {
+        logSubprocessOutput(
+            command, Files.readString(forkedProcess.getProcessOutputFile(), charset));
+        throw t;
+      }
+    }
+  }
+
+  protected void logSubprocessOutput(String command, String output) {
+    System.out.printf(
+        Locale.ROOT,
+        "--- [forked subprocess output: %s] ---%n%s%n--- [end of subprocess output] ---%n",
+        command,
+        output);
+  }
+}
diff --git a/lucene/distribution/src/binary-release/bin/luke.cmd b/lucene/distribution/src/binary-release/bin/luke.cmd
index ee5d06c..9c20710 100644
--- a/lucene/distribution/src/binary-release/bin/luke.cmd
+++ b/lucene/distribution/src/binary-release/bin/luke.cmd
@@ -17,5 +17,15 @@
 
 SETLOCAL
 SET MODULES=%~dp0..
-start javaw --module-path "%MODULES%\modules;%MODULES%\modules-thirdparty" --module org.apache.lucene.luke
+
+REM For distribution testing we want plain 'java' command, otherwise we can't block
+REM on luke invocation and can't intercept the return status.
+SET LAUNCH_CMD=start javaw
+IF NOT "%DISTRIBUTION_TESTING%"=="true" GOTO launch
+SET LAUNCH_CMD=java
+
+:launch
+%LAUNCH_CMD% --module-path "%MODULES%\modules;%MODULES%\modules-thirdparty" --module org.apache.lucene.luke %*
+SET EXITVAL=%errorlevel%
+EXIT /b %EXITVAL%
 ENDLOCAL
diff --git a/lucene/distribution/src/binary-release/bin/luke.sh b/lucene/distribution/src/binary-release/bin/luke.sh
index b934f4e..b2d7d5e 100644
--- a/lucene/distribution/src/binary-release/bin/luke.sh
+++ b/lucene/distribution/src/binary-release/bin/luke.sh
@@ -17,4 +17,5 @@
 
 MODULES=`dirname "$0"`/..
 MODULES=`cd "$MODULES" && pwd`
-java --module-path "$MODULES/modules:$MODULES/modules-thirdparty" --module org.apache.lucene.luke
+java --module-path "$MODULES/modules:$MODULES/modules-thirdparty" --module org.apache.lucene.luke "$@"
+exit $?
diff --git a/lucene/licenses/procfork-1.0.6.jar.sha1 b/lucene/licenses/procfork-1.0.6.jar.sha1
new file mode 100644
index 0000000..7d6b882
--- /dev/null
+++ b/lucene/licenses/procfork-1.0.6.jar.sha1
@@ -0,0 +1 @@
+7ac0dae744df9cc3aaa7a5fee72e289cad7790f9
diff --git a/lucene/licenses/procfork-LICENSE-ASL.txt b/lucene/licenses/procfork-LICENSE-ASL.txt
new file mode 100644
index 0000000..ba60654
--- /dev/null
+++ b/lucene/licenses/procfork-LICENSE-ASL.txt
@@ -0,0 +1,34 @@
+
+Copyright (C) 2019 Dawid Weiss
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+- Redistributions of  source code must  retain the above  copyright notice, this
+  list of conditions and the following disclaimer. 
+
+- Redistributions in binary form must reproduce the above copyright notice, this
+  list of conditions and the following  disclaimer in  the documentation  and/or
+  other materials provided with the distribution. 
+
+- Neither the name  of the Carrot2 Project  nor  the names  of  its contributors 
+  may  be used  to endorse  or  promote  products derived   from  this  software 
+  without specific prior written permission.
+
+- We kindly request that you include in the end-user documentation provided with
+  the redistribution and/or in the software itself an acknowledgement equivalent 
+  to  the  following:  "This product includes  software developed by the Carrot2 
+  Project."
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"  AND
+ANY EXPRESS OR  IMPLIED WARRANTIES, INCLUDING,  BUT NOT LIMITED  TO, THE IMPLIED
+WARRANTIES  OF  MERCHANTABILITY  AND  FITNESS  FOR  A  PARTICULAR  PURPOSE   ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE  FOR
+ANY DIRECT, INDIRECT, INCIDENTAL,  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL  DAMAGES
+(INCLUDING, BUT  NOT LIMITED  TO, PROCUREMENT  OF SUBSTITUTE  GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS;  OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND  ON
+ANY  THEORY  OF  LIABILITY,  WHETHER  IN  CONTRACT,  STRICT  LIABILITY,  OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE)  ARISING IN ANY WAY  OUT OF THE USE  OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
diff --git a/lucene/licenses/procfork-NOTICE.txt b/lucene/licenses/procfork-NOTICE.txt
new file mode 100644
index 0000000..e69de29
diff --git a/lucene/luke/src/java/org/apache/lucene/luke/app/desktop/LukeMain.java b/lucene/luke/src/java/org/apache/lucene/luke/app/desktop/LukeMain.java
index cd78468..0d52fd9 100644
--- a/lucene/luke/src/java/org/apache/lucene/luke/app/desktop/LukeMain.java
+++ b/lucene/luke/src/java/org/apache/lucene/luke/app/desktop/LukeMain.java
@@ -19,13 +19,13 @@ package org.apache.lucene.luke.app.desktop;
 
 import static org.apache.lucene.luke.app.desktop.util.ExceptionHandler.handle;
 
-import java.awt.GraphicsEnvironment;
+import java.awt.*;
 import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
 import java.util.concurrent.SynchronousQueue;
 import java.util.logging.Level;
 import java.util.logging.Logger;
-import javax.swing.JFrame;
-import javax.swing.UIManager;
+import javax.swing.*;
 import org.apache.lucene.luke.app.desktop.components.LukeWindowProvider;
 import org.apache.lucene.luke.app.desktop.components.dialog.menubar.OpenIndexDialogFactory;
 import org.apache.lucene.luke.app.desktop.util.DialogOpener;
@@ -49,7 +49,7 @@ public class LukeMain {
   }
 
   /** @return Returns {@code true} if GUI startup and initialization was successful. */
-  private static boolean createAndShowGUI() {
+  private static boolean createGUI() {
     // uncaught error handler
     MessageBroker messageBroker = MessageBroker.getInstance();
     try {
@@ -61,14 +61,6 @@ public class LukeMain {
       frame.pack();
       frame.setVisible(true);
 
-      OpenIndexDialogFactory openIndexDialogFactory = OpenIndexDialogFactory.getInstance();
-      new DialogOpener<>(openIndexDialogFactory)
-          .open(
-              MessageUtils.getLocalizedMessage("openindex.dialog.title"),
-              600,
-              420,
-              (factory) -> {});
-
       return true;
     } catch (Throwable e) {
       messageBroker.showUnknownErrorMessage();
@@ -78,6 +70,13 @@ public class LukeMain {
   }
 
   public static void main(String[] args) throws Exception {
+    boolean sanityCheck = Arrays.asList(args).contains("--sanity-check");
+
+    if (sanityCheck && GraphicsEnvironment.isHeadless()) {
+      Logger.getGlobal().log(Level.SEVERE, "[Vader] Hello, Luke. Can't do much in headless mode.");
+      Runtime.getRuntime().exit(0);
+    }
+
     String lookAndFeelClassName = UIManager.getSystemLookAndFeelClassName();
     if (!lookAndFeelClassName.contains("AquaLookAndFeel")
         && !lookAndFeelClassName.contains("PlasticXPLookAndFeel")) {
@@ -93,8 +92,17 @@ public class LukeMain {
     javax.swing.SwingUtilities.invokeLater(
         () -> {
           try {
-            guiThreadResult.put(createAndShowGUI());
-          } catch (InterruptedException e) {
+            guiThreadResult.put(createGUI());
+
+            // Show the initial dialog.
+            OpenIndexDialogFactory openIndexDialogFactory = OpenIndexDialogFactory.getInstance();
+            new DialogOpener<>(openIndexDialogFactory)
+                .open(
+                    MessageUtils.getLocalizedMessage("openindex.dialog.title"),
+                    600,
+                    420,
+                    (factory) -> {});
+          } catch (Exception e) {
             throw new RuntimeException(e);
           }
         });
@@ -103,5 +111,11 @@ public class LukeMain {
       Logger.getGlobal().log(Level.SEVERE, "Luke could not start.");
       Runtime.getRuntime().exit(1);
     }
+
+    if (sanityCheck) {
+      // In sanity-check mode on non-headless displays, return success.
+      Logger.getGlobal().log(Level.SEVERE, "[Vader] Hello, Luke. We seem to be fine.");
+      Runtime.getRuntime().exit(0);
+    }
   }
 }
diff --git a/versions.lock b/versions.lock
index fb12bb6..b7fa511 100644
--- a/versions.lock
+++ b/versions.lock
@@ -26,6 +26,7 @@ ua.net.nlp:morfologik-ukrainian-search:4.9.1 (1 constraints: 10051b36)
 xerces:xercesImpl:2.12.0 (1 constraints: 3705353b)
 
 [Test dependencies]
+com.carrotsearch:procfork:1.0.6 (1 constraints: 0905f635)
 org.assertj:assertj-core:3.21.0 (1 constraints: 38053c3b)
 org.eclipse.jetty:jetty-continuation:9.4.41.v20210516 (1 constraints: 7907fe7c)
 org.eclipse.jetty:jetty-http:9.4.41.v20210516 (1 constraints: f60f2ccd)
diff --git a/versions.props b/versions.props
index 176f122..b2e69b2 100644
--- a/versions.props
+++ b/versions.props
@@ -1,5 +1,6 @@
 com.carrotsearch.randomizedtesting:*=2.7.6
 com.carrotsearch:hppc=0.9.0
+com.carrotsearch:procfork=1.0.6
 com.google.errorprone:*=2.10.0
 com.ibm.icu:icu4j=70.1
 commons-codec:commons-codec=1.13