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 2020/09/09 17:04:32 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9512: Move LockFactory stress test to be a unit/integration test (#1842)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new c319502  LUCENE-9512: Move LockFactory stress test to be a unit/integration test (#1842)
c319502 is described below

commit c319502bd5d570d2df2d1b53b3efae38d42c41ba
Author: Uwe Schindler <us...@apache.org>
AuthorDate: Wed Sep 9 18:42:30 2020 +0200

    LUCENE-9512: Move LockFactory stress test to be a unit/integration test (#1842)
    
    Co-authored-by: Dawid Weiss <da...@carrotsearch.com>
---
 lucene/CHANGES.txt                                 |   3 +
 lucene/core/build.xml                              |  88 -----------------
 .../org/apache/lucene/store/LockStressTest.java    |  43 +++++----
 .../org/apache/lucene/store/LockVerifyServer.java  |  44 ++++-----
 .../apache/lucene/store/VerifyingLockFactory.java  |   6 +-
 .../lucene/store/TestStressLockFactories.java      | 104 +++++++++++++++++++++
 lucene/tools/junit4/tests.policy                   |   3 +
 7 files changed, 160 insertions(+), 131 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index ea9c08f..400c00a 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -82,6 +82,9 @@ Other
 
 * LUCENE-9470: Make TestXYMultiPolygonShapeQueries more resilient for CONTAINS queries. (Ignacio Vera)
 
+* LUCENE-9512: Move LockFactory stress test to be a unit/integration
+  test. (Uwe Schindler, Dawid Weiss, Robert Muir)
+
 ======================= Lucene 8.6.2 =======================
 
 Bug Fixes
diff --git a/lucene/core/build.xml b/lucene/core/build.xml
index 01853b6..a7160b0 100644
--- a/lucene/core/build.xml
+++ b/lucene/core/build.xml
@@ -145,94 +145,6 @@
 
   <target name="regenerate" depends="createLevAutomata,createPackedIntSources,jflex"/>
   
-  <macrodef name="startLockStressTestClient">
-    <attribute name="clientId"/>
-    <attribute name="lockFactoryImpl"/>
-    <attribute name="lockFactoryDir"/>
-    <sequential>
-      <local name="lockverifyserver.port"/>
-      <groovy><![CDATA[
-        String port;
-        while ((port = System.getProperty("lockverifyserver.port")) == null) {
-          Thread.sleep(10L);
-        }
-        properties["lockverifyserver.port"] = port;
-      ]]></groovy>
-      <java taskname="lockStressTest@{clientId}" fork="true" classpathref="test-lock.classpath" classname="org.apache.lucene.store.LockStressTest" failOnError="true"> 
-        <arg value="@{clientId}"/>
-        <arg value="${lockverifyserver.host}"/>
-        <arg value="${lockverifyserver.port}"/>
-        <arg value="@{lockFactoryImpl}"/>
-        <arg value="@{lockFactoryDir}"/>
-        <arg value="${lockverify.delay}"/>
-        <arg value="${lockverify.count}"/>
-      </java>
-    </sequential>
-  </macrodef>
-  
-  <macrodef name="testLockFactory">
-    <attribute name="lockFactoryImpl"/>
-    <attribute name="lockFactoryDir"/>
-    <sequential>
-      <echo taskname="testLockFactory" message="Testing @{lockFactoryImpl}..."/>
-      <mkdir dir="@{lockFactoryDir}"/>
-      <parallel threadCount="3" failonany="false">
-        <sequential>
-          <!-- the server runs in-process, so we can wait for the sysproperty -->
-          <java taskname="lockVerifyServer" fork="false" classpathref="test-lock.classpath" classname="org.apache.lucene.store.LockVerifyServer" failOnError="true">
-            <arg value="${lockverifyserver.host}"/>
-            <arg value="2"/>
-          </java>
-        </sequential>
-        <sequential>
-          <startLockStressTestClient clientId="1" lockFactoryImpl="@{lockFactoryImpl}" lockFactoryDir="@{lockFactoryDir}" />
-        </sequential>
-        <sequential>
-          <startLockStressTestClient clientId="2" lockFactoryImpl="@{lockFactoryImpl}" lockFactoryDir="@{lockFactoryDir}" />
-        </sequential>
-      </parallel>
-    </sequential>
-  </macrodef>
-  
-  <condition property="-ignore-test-lock-factory">
-    <or>
-      <!-- We ignore our ant-based lock factory test, if user applies test filtering: -->
-      <isset property="tests.class" />
-      <isset property="tests.method" />
-      <!-- Clover seems to deadlock if running instrumented code inside the Ant JVM: -->
-      <isset property="run.clover" />
-    </or>
-  </condition>
-  
-  <target name="test-lock-factory" depends="resolve-groovy,compile-core" unless="-ignore-test-lock-factory"
-    description="Run LockStressTest with multiple JVMs">
-    <property name="lockverifyserver.host" value="127.0.0.1"/>
-    <property name="lockverify.delay" value="1"/>
-    <groovy taskname="lockVerifySetup"><![CDATA[
-      System.clearProperty("lockverifyserver.port"); // make sure it is undefined
-      
-      if (!properties["lockverify.count"]) {
-        int count = Boolean.parseBoolean(properties["tests.nightly"]) ?
-          30000 : 500;
-        count *= Integer.parseInt(properties["tests.multiplier"]);
-        properties["lockverify.count"] = count;
-      }
-      
-      task.log("Configuration properties:");
-      ["lockverify.delay", "lockverify.count"].each {
-        k -> task.log(" " + k + "=" + properties[k]);
-      }
-    ]]></groovy>
-    <path id="test-lock.classpath">
-      <path refid="classpath"/>
-      <pathelement location="${build.dir}/classes/java"/>
-    </path>
-    <testLockFactory lockFactoryImpl="org.apache.lucene.store.NativeFSLockFactory" lockFactoryDir="${build.dir}/lockfactorytest/native" />
-    <testLockFactory lockFactoryImpl="org.apache.lucene.store.SimpleFSLockFactory" lockFactoryDir="${build.dir}/lockfactorytest/simple" />
-  </target>
-  
-  <target name="test" depends="common.test, test-lock-factory"/>
-
   <target name="clean-jflex">
     <delete>
       <fileset dir="src/java/org/apache/lucene/analysis/standard" includes="**/*.java">
diff --git a/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java b/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java
index 2008f91..bfca637 100644
--- a/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java
+++ b/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java
@@ -30,7 +30,7 @@ import org.apache.lucene.util.SuppressForbidden;
 
 /**
  * Simple standalone tool that forever acquires and releases a
- * lock using a specific LockFactory.  Run without any args
+ * lock using a specific {@link LockFactory}.  Run without any args
  * to see usage.
  *
  * @see VerifyingLockFactory
@@ -38,11 +38,9 @@ import org.apache.lucene.util.SuppressForbidden;
  */ 
 
 public class LockStressTest {
-  
   static final String LOCK_FILE_NAME = "test.lock";
-
+  
   @SuppressForbidden(reason = "System.out required: command line tool")
-  @SuppressWarnings("try")
   public static void main(String[] args) throws Exception {
     if (args.length != 7) {
       System.out.println("Usage: java org.apache.lucene.store.LockStressTest myID verifierHost verifierPort lockFactoryClassName lockDirName sleepTimeMS count\n" +
@@ -65,12 +63,6 @@ public class LockStressTest {
 
     int arg = 0;
     final int myID = Integer.parseInt(args[arg++]);
-
-    if (myID < 0 || myID > 255) {
-      System.out.println("myID must be a unique int 0..255");
-      System.exit(1);
-    }
-
     final String verifierHost = args[arg++];
     final int verifierPort = Integer.parseInt(args[arg++]);
     final String lockFactoryClassName = args[arg++];
@@ -78,8 +70,21 @@ public class LockStressTest {
     final int sleepTimeMS = Integer.parseInt(args[arg++]);
     final int count = Integer.parseInt(args[arg++]);
 
+    int exitCode = run(myID, verifierHost, verifierPort, lockFactoryClassName, lockDirPath, sleepTimeMS, count);
+    System.exit(exitCode);
+  }
+
+  @SuppressForbidden(reason = "System.out required: command line tool")
+  @SuppressWarnings("try")
+  private static int run(int myID, String verifierHost, int verifierPort, String lockFactoryClassName,
+                        Path lockDirPath, int sleepTimeMS, int count) throws IOException, InterruptedException {
+    if (myID < 0 || myID > 255) {
+      System.out.println("myID must be a unique int 0..255");
+      return 1;
+    }
+
     final LockFactory lockFactory = getNewLockFactory(lockFactoryClassName);
-    // we test the lock factory directly, so we don't need it on the directory itsself (the directory is just for testing)
+    // we test the lock factory directly, so we don't need it on the directory itself (the directory is just for testing)
     final FSDirectory lockDir = new SimpleFSDirectory(lockDirPath, NoLockFactory.INSTANCE);
     final InetSocketAddress addr = new InetSocketAddress(verifierHost, verifierPort);
     System.out.println("Connecting to server " + addr +
@@ -89,17 +94,17 @@ public class LockStressTest {
       socket.connect(addr, 500);
       final OutputStream out = socket.getOutputStream();
       final InputStream in = socket.getInputStream();
-      
+
       out.write(myID);
       out.flush();
       LockFactory verifyLF = new VerifyingLockFactory(lockFactory, in, out);
       final Random rnd = new Random();
-      
+
       // wait for starting gun
-      if (in.read() != 43) {
+      if (in.read() != LockVerifyServer.START_GUN_SIGNAL) {
         throw new IOException("Protocol violation");
       }
-      
+
       for (int i = 0; i < count; i++) {
         try (final Lock l = verifyLF.obtainLock(lockDir, LOCK_FILE_NAME)) {
           if (rnd.nextInt(10) == 0) {
@@ -120,15 +125,15 @@ public class LockStressTest {
         if (i % 500 == 0) {
           System.out.println((i * 100. / count) + "% done.");
         }
-        
+
         Thread.sleep(sleepTimeMS);
-      } 
+      }
     }
-    
+
     System.out.println("Finished " + count + " tries.");
+    return 0;
   }
 
-
   private static FSLockFactory getNewLockFactory(String lockFactoryClassName) throws IOException {
     // try to get static INSTANCE field of class
     try {
diff --git a/lucene/core/src/java/org/apache/lucene/store/LockVerifyServer.java b/lucene/core/src/java/org/apache/lucene/store/LockVerifyServer.java
index 3ce0c06..c9d77bb 100644
--- a/lucene/core/src/java/org/apache/lucene/store/LockVerifyServer.java
+++ b/lucene/core/src/java/org/apache/lucene/store/LockVerifyServer.java
@@ -24,6 +24,7 @@ import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.util.concurrent.CountDownLatch;
+import java.util.function.Consumer;
 
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.SuppressForbidden;
@@ -38,20 +39,12 @@ import org.apache.lucene.util.SuppressForbidden;
  * @see LockStressTest
  */
 
+@SuppressForbidden(reason = "System.out required: command line tool")
 public class LockVerifyServer {
+  public static final int START_GUN_SIGNAL = 43;
 
-  @SuppressForbidden(reason = "System.out required: command line tool")
-  public static void main(String[] args) throws Exception {
-
-    if (args.length != 2) {
-      System.out.println("Usage: java org.apache.lucene.store.LockVerifyServer bindToIp clients\n");
-      System.exit(1);
-    }
-
-    int arg = 0;
-    final String hostname = args[arg++];
-    final int maxClients = Integer.parseInt(args[arg++]);
-
+  // method pkg-private for tests
+  static void run(String hostname, int maxClients, Consumer<InetSocketAddress> startClients) throws Exception {
     try (final ServerSocket s = new ServerSocket()) {
       s.setReuseAddress(true);
       s.setSoTimeout(30000); // initially 30 secs to give clients enough time to startup
@@ -59,8 +52,8 @@ public class LockVerifyServer {
       final InetSocketAddress localAddr = (InetSocketAddress) s.getLocalSocketAddress();
       System.out.println("Listening on " + localAddr + "...");
       
-      // we set the port as a sysprop, so the ANT task can read it. For that to work, this server must run in-process:
-      System.setProperty("lockverifyserver.port", Integer.toString(localAddr.getPort()));
+      // callback only for the test to start the clients:
+      startClients.accept(localAddr);
       
       final Object localLock = new Object();
       final int[] lockedID = new int[1];
@@ -80,22 +73,22 @@ public class LockVerifyServer {
               }
               
               startingGun.await();
-              os.write(43);
+              os.write(START_GUN_SIGNAL);
               os.flush();
               
-              while(true) {
+              while (true) {
                 final int command = in.read();
                 if (command < 0) {
                   return; // closed
                 }
                 
-                synchronized(localLock) {
+                synchronized (localLock) {
                   final int currentLock = lockedID[0];
                   if (currentLock == -2) {
                     return; // another thread got error, so we exit, too!
                   }
                   switch (command) {
-                    case 1:
+                    case VerifyingLockFactory.MSG_LOCK_ACQUIRED:
                       // Locked
                       if (currentLock != -1) {
                         lockedID[0] = -2;
@@ -103,7 +96,7 @@ public class LockVerifyServer {
                       }
                       lockedID[0] = id;
                       break;
-                    case 0:
+                    case VerifyingLockFactory.MSG_LOCK_RELEASED:
                       // Unlocked
                       if (currentLock != id) {
                         lockedID[0] = -2;
@@ -138,11 +131,18 @@ public class LockVerifyServer {
       for (Thread t : threads) {
         t.join();
       }
-      
-      // cleanup sysprop
-      System.clearProperty("lockverifyserver.port");
 
       System.out.println("Server terminated.");
     }
   }
+
+  public static void main(String[] args) throws Exception {
+    if (args.length != 2) {
+      System.out.println("Usage: java org.apache.lucene.store.LockVerifyServer bindToIp clients\n");
+      System.exit(1);
+    }
+
+    run(args[0], Integer.parseInt(args[1]), addr -> {});
+  }
+  
 }
diff --git a/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java b/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java
index 6a7214b..1b9ce23 100644
--- a/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java
@@ -36,6 +36,8 @@ import java.io.OutputStream;
  */
 
 public final class VerifyingLockFactory extends LockFactory {
+  public static final int MSG_LOCK_RELEASED = 0;
+  public static final int MSG_LOCK_ACQUIRED = 1;
 
   final LockFactory lf;
   final InputStream in;
@@ -46,7 +48,7 @@ public final class VerifyingLockFactory extends LockFactory {
 
     public CheckedLock(Lock lock) throws IOException {
       this.lock = lock;
-      verify((byte) 1);
+      verify((byte) MSG_LOCK_ACQUIRED);
     }
 
     @Override
@@ -58,7 +60,7 @@ public final class VerifyingLockFactory extends LockFactory {
     public void close() throws IOException {
       try (Lock l = lock) {
         l.ensureValid();
-        verify((byte) 0);
+        verify((byte) MSG_LOCK_RELEASED);
       }
     }
 
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestStressLockFactories.java b/lucene/core/src/test/org/apache/lucene/store/TestStressLockFactories.java
new file mode 100644
index 0000000..041b00b
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/store/TestStressLockFactories.java
@@ -0,0 +1,104 @@
+/*
+ * 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.ProcessBuilder.Redirect;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.SuppressForbidden;
+
+@LuceneTestCase.SuppressFileSystems("*")
+public class TestStressLockFactories extends LuceneTestCase {
+  
+  @SuppressForbidden(reason = "ProcessBuilder only allows to redirect to java.io.File")
+  private static final ProcessBuilder applyRedirection(ProcessBuilder pb, int client, Path dir) {
+    if (VERBOSE) {
+      return pb.inheritIO();
+    } else {
+      return pb
+        .redirectError(dir.resolve("err-" + client + ".txt").toFile())
+        .redirectOutput(dir.resolve("out-" + client + ".txt").toFile())
+        .redirectInput(Redirect.INHERIT);
+    }
+  }
+  
+  private void runImpl(Class<? extends LockFactory> impl) throws Exception {
+    final int clients = TEST_NIGHTLY ? 5 : 2;
+    final String host = "127.0.0.1";
+    final int delay = 1;
+    final int rounds = (TEST_NIGHTLY ? 30000 : 500) * RANDOM_MULTIPLIER;
+    
+    final Path dir = createTempDir(impl.getSimpleName());
+    
+    final List<Process> processes = new ArrayList<>(clients);
+
+    LockVerifyServer.run(host, clients, addr -> {
+      // spawn clients as separate Java processes
+      for (int i = 0; i < clients; i++) {
+        try {
+          processes.add(applyRedirection(new ProcessBuilder(
+              Paths.get(System.getProperty("java.home"), "bin", "java").toString(),
+              "-Xmx32M",
+              "-cp",
+              System.getProperty("java.class.path"),
+              LockStressTest.class.getName(),
+              Integer.toString(i),
+              addr.getHostString(),
+              Integer.toString(addr.getPort()),
+              impl.getName(),
+              dir.toString(),
+              Integer.toString(delay),
+              Integer.toString(rounds)
+            ), i, dir).start());
+        } catch (IOException ioe) {
+          throw new AssertionError("Failed to start child process.", ioe);
+        }
+      }
+    });
+     
+    // wait for all processes to exit...
+    try {
+      for (Process p : processes) {
+        if (p.waitFor(15, TimeUnit.SECONDS)) {
+          assertEquals("Process died abnormally?", 0, p.waitFor());
+        }
+      }
+    } finally {
+      // kill all processes, which are still alive.
+      for (Process p : processes) {
+        if (p.isAlive()) {
+          p.destroyForcibly().waitFor();
+        }
+      }
+    }
+  }
+  
+  public void testNativeFSLockFactory() throws Exception {
+    runImpl(NativeFSLockFactory.class);
+  }
+
+  public void testSimpleFSLockFactory() throws Exception {
+    runImpl(SimpleFSLockFactory.class);
+  }
+
+}
diff --git a/lucene/tools/junit4/tests.policy b/lucene/tools/junit4/tests.policy
index 7494981..37cc9e8 100644
--- a/lucene/tools/junit4/tests.policy
+++ b/lucene/tools/junit4/tests.policy
@@ -71,6 +71,9 @@ grant {
   permission java.lang.RuntimePermission "getClassLoader";
   permission java.lang.RuntimePermission "setContextClassLoader";
 
+  // TestLockFactoriesMultiJVM opens a random port on 127.0.0.1 (port 0 = ephemeral port range):
+  permission java.net.SocketPermission "127.0.0.1:0", "accept,listen,resolve";
+
   // read access to all system properties:
   permission java.util.PropertyPermission "*", "read";
   // write access to only these: