You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2012/06/29 10:07:58 UTC

svn commit: r1355258 - in /lucene/dev/trunk/lucene: ./ core/src/test/org/apache/lucene/util/ test-framework/src/java/org/apache/lucene/util/

Author: dweiss
Date: Fri Jun 29 08:07:56 2012
New Revision: 1355258

URL: http://svn.apache.org/viewvc?rev=1355258&view=rev
Log:
LUCENE-4160: Bring back the functional equivalent of tests.iters.min. Added two properties: -Dtests.maxfailures=M and -Dtests.failfast=[true/false/yes/no/on/off] which control skipping tests after the first M failures have been hit or after the first failure (failfast is an alias for tests.maxfailures=1).

Added:
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/common-build.xml
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreTestSuites.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleMarkFailure.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1355258&r1=1355257&r2=1355258&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri Jun 29 08:07:56 2012
@@ -24,6 +24,13 @@ Bug Fixes
 
 Build
 
+* LUCENE-4160: Added a property to quit the tests after a given
+  number of failures has occurred. This is useful in combination
+  with -Dtests.iters=N (you can start N iterations and wait for M
+  failures, in particular M = 1). -Dtests.maxfailures=M. Alternatively,
+  specify -Dtests.failfast=true to skip all tests after the first failure.
+  (Dawid Weiss)
+
 * LUCENE-4115: JAR resolution/ cleanup should be done automatically for ant 
   clean/ eclipse/ resolve (Dawid Weiss)
 

Modified: lucene/dev/trunk/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/common-build.xml?rev=1355258&r1=1355257&r2=1355258&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/common-build.xml (original)
+++ lucene/dev/trunk/lucene/common-build.xml Fri Jun 29 08:07:56 2012
@@ -818,6 +818,12 @@
             <sysproperty key="jetty.insecurerandom" value="1"/>
             <sysproperty key="solr.directoryFactory" value="org.apache.solr.core.MockDirectoryFactory"/>
 
+            <!-- Only pass these to the test JVMs if defined in ANT. -->
+            <syspropertyset>
+                <propertyref prefix="tests.maxfailures" />
+                <propertyref prefix="tests.failfast" />
+            </syspropertyset>
+
             <!-- Use static cached test balancing statistcs. -->
             <balancers>
                 <execution-times>
@@ -961,6 +967,10 @@ ant test -Dtests.iters=N -Dtestcase=Clas
 ant test -Dtests.iters=N -Dtestcase=ClassName -Dtestmethod=mytest
 ant test -Dtests.iters=N -Dtestcase=ClassName -Dtests.method=mytest*
 
+# Repeats N times but skips any tests after the first failure or M
+# initial failures.
+ant test -Dtests.iters=N -Dtests.failfast=yes -Dtestcase=...
+ant test -Dtests.iters=N -Dtests.maxfailures=M -Dtestcase=...
 
 #
 # Test groups. ----------------------------------------------------

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java?rev=1355258&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java Fri Jun 29 08:07:56 2012
@@ -0,0 +1,72 @@
+package org.apache.lucene.util;
+
+/*
+ * 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 org.apache.lucene.util.junitcompat.WithNestedTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.*;
+import org.junit.runner.notification.Failure;
+import org.junit.runner.notification.RunListener;
+
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
+
+/**
+ * @see TestRuleIgnoreAfterMaxFailures
+ * @see SystemPropertiesInvariantRule
+ */
+public class TestMaxFailuresRule extends WithNestedTests {
+  public TestMaxFailuresRule() {
+    super(true);
+  }
+
+  public static class Nested extends WithNestedTests.AbstractNestedTest {
+    @Repeat(iterations = 100)
+    public void testFailSometimes() {
+      assertFalse(random().nextInt(5) == 0);
+    }
+  }
+
+  @Test
+  public void testMaxFailures() {
+    int maxFailures = LuceneTestCase.ignoreAfterMaxFailures.getMaxFailures();
+    try {
+      LuceneTestCase.ignoreAfterMaxFailures.setMaxFailures(2);
+
+      JUnitCore core = new JUnitCore();
+      final int [] assumptions = new int [1];
+      core.addListener(new RunListener() {
+        @Override
+        public void testAssumptionFailure(Failure failure) {
+          assumptions[0]++; 
+        }
+      });
+
+      Result result = core.run(Nested.class);
+      Assert.assertEquals(2, result.getFailureCount());
+      Assert.assertEquals(0, result.getIgnoreCount());
+      Assert.assertEquals(100, result.getRunCount());
+      // JUnit doesn't pass back the number of successful tests, just make sure
+      // we did have enough assumption-failures.
+      Assert.assertTrue(assumptions[0] > 50);
+    } finally {
+      LuceneTestCase.ignoreAfterMaxFailures.setMaxFailures(maxFailures);
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1355258&r1=1355257&r2=1355258&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Fri Jun 29 08:07:56 2012
@@ -23,6 +23,7 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.util.*;
 import java.util.concurrent.*;
+import java.util.logging.Logger;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Field.Store;
@@ -115,15 +116,21 @@ import static com.carrotsearch.randomize
 @ThreadLeaks(failTestIfLeaking = false)
 public abstract class LuceneTestCase extends Assert {
 
-  // -----------------------------------------------------------------
-  // Test groups and other annotations modifying tests' behavior.
-  // -----------------------------------------------------------------
-   
+  // --------------------------------------------------------------------
+  // Test groups, system properties and other annotations modifying tests
+  // --------------------------------------------------------------------
+
   public static final String SYSPROP_NIGHTLY = "tests.nightly";
   public static final String SYSPROP_WEEKLY = "tests.weekly";
   public static final String SYSPROP_AWAITSFIX = "tests.awaitsfix";
   public static final String SYSPROP_SLOW = "tests.slow";
 
+  /** @see #ignoreAfterMaxFailures*/
+  private static final String SYSPROP_MAXFAILURES = "tests.maxfailures";
+
+  /** @see #ignoreAfterMaxFailures*/
+  private static final String SYSPROP_FAILFAST = "tests.failfast";
+
   /**
    * Annotation for tests that should only be run during nightly builds.
    */
@@ -232,7 +239,7 @@ public abstract class LuceneTestCase ext
 
   /** Whether or not {@link Slow} tests should run. */
   public static final boolean TEST_SLOW = systemPropertyAsBoolean(SYSPROP_SLOW, false);
-  
+
   /** Throttling, see {@link MockDirectoryWrapper#setThrottling(Throttling)}. */
   public static final Throttling TEST_THROTTLING = TEST_NIGHTLY ? Throttling.SOMETIMES : Throttling.NEVER;
 
@@ -298,8 +305,30 @@ public abstract class LuceneTestCase ext
   /**
    * Suite failure marker (any error in the test or suite scope).
    */
-  public static TestRuleMarkFailure suiteFailureMarker;
-  
+  public final static TestRuleMarkFailure suiteFailureMarker = 
+      new TestRuleMarkFailure();
+
+  /**
+   * Ignore tests after hitting a designated number of initial failures.
+   */
+  final static TestRuleIgnoreAfterMaxFailures ignoreAfterMaxFailures; 
+  static {
+    int maxFailures = systemPropertyAsInt(SYSPROP_MAXFAILURES, Integer.MAX_VALUE);
+    boolean failFast = systemPropertyAsBoolean(SYSPROP_FAILFAST, false);
+
+    if (failFast) {
+      if (maxFailures == Integer.MAX_VALUE) {
+        maxFailures = 1;
+      } else {
+        Logger.getLogger(LuceneTestCase.class.getSimpleName()).warning(
+            "Property '" + SYSPROP_MAXFAILURES + "'=" + maxFailures + ", 'failfast' is" +
+            		" ignored.");
+      }
+    }
+
+    ignoreAfterMaxFailures = new TestRuleIgnoreAfterMaxFailures(maxFailures);
+  }
+
   /**
    * This controls how suite-level rules are nested. It is important that _all_ rules declared
    * in {@link LuceneTestCase} are executed in proper order if they depend on each 
@@ -308,7 +337,8 @@ public abstract class LuceneTestCase ext
   @ClassRule
   public static TestRule classRules = RuleChain
     .outerRule(new TestRuleIgnoreTestSuites())
-    .around(suiteFailureMarker = new TestRuleMarkFailure())
+    .around(ignoreAfterMaxFailures)
+    .around(suiteFailureMarker)
     .around(new TestRuleAssertionsRequired())
     .around(new TestRuleNoStaticHooksShadowing())
     .around(new TestRuleNoInstanceHooksOverrides())
@@ -329,7 +359,7 @@ public abstract class LuceneTestCase ext
   /** Save test thread and name. */
   private TestRuleThreadAndTestName threadAndTestNameRule = new TestRuleThreadAndTestName();
 
-  /** Taint test failures. */
+  /** Taint suite result with individual test failures. */
   private TestRuleMarkFailure testFailureMarker = new TestRuleMarkFailure(suiteFailureMarker); 
   
   /**
@@ -340,6 +370,7 @@ public abstract class LuceneTestCase ext
   @Rule
   public final TestRule ruleChain = RuleChain
     .outerRule(testFailureMarker)
+    .around(ignoreAfterMaxFailures)
     .around(threadAndTestNameRule)
     .around(new TestRuleReportUncaughtExceptions())
     .around(new SystemPropertiesInvariantRule(IGNORED_INVARIANT_PROPERTIES))

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java?rev=1355258&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java Fri Jun 29 08:07:56 2012
@@ -0,0 +1,90 @@
+package org.apache.lucene.util;
+
+import org.junit.Assert;
+import org.junit.internal.AssumptionViolatedException;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+import com.carrotsearch.randomizedtesting.RandomizedTest;
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
+
+/*
+ * 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.
+ */
+
+/**
+ * This rule keeps a count of failed tests (suites) and will result in an
+ * {@link AssumptionViolatedException} after a given number of failures for all
+ * tests following this condition.
+ * 
+ * <p>
+ * Aborting quickly on failed tests can be useful when used in combination with
+ * test repeats (via the {@link Repeat} annotation or system property).
+ */
+public final class TestRuleIgnoreAfterMaxFailures implements TestRule {
+  /**
+   * Maximum failures.
+   */
+  private int maxFailures;
+
+  /**
+   * Current count of failures.
+   */
+  private int failuresSoFar;
+  
+  /**
+   * @param maxFailures
+   *          The number of failures after which all tests are ignored. Must be
+   *          greater or equal 1.
+   */
+  public TestRuleIgnoreAfterMaxFailures(int maxFailures) {
+    Assert.assertTrue("maxFailures must be >= 1: " + maxFailures, maxFailures >= 1);
+    this.maxFailures = maxFailures;
+  }
+
+  @Override
+  public Statement apply(final Statement s, final Description d) {
+    return new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+        if (failuresSoFar >= maxFailures) {
+          RandomizedTest.assumeTrue("Ignored, failures limit reached (" + 
+              failuresSoFar + " >= " + maxFailures + ").", false);
+        }
+
+        try {
+          s.evaluate();
+        } catch (Throwable t) {
+          if (!TestRuleMarkFailure.isAssumption(t)) {
+            System.out.println("#" + d);
+            failuresSoFar++;
+          }
+          throw t;
+        }
+      }
+    };
+  }
+
+  /** For tests only. */
+  void setMaxFailures(int maxFailures) {
+    this.maxFailures = maxFailures;
+  }
+  
+  int getMaxFailures() {
+    return maxFailures;
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreTestSuites.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreTestSuites.java?rev=1355258&r1=1355257&r2=1355258&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreTestSuites.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreTestSuites.java Fri Jun 29 08:07:56 2012
@@ -1,6 +1,5 @@
 package org.apache.lucene.util;
 
-import org.junit.Assume;
 import org.junit.rules.TestRule;
 import org.junit.runner.Description;
 import org.junit.runners.model.Statement;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleMarkFailure.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleMarkFailure.java?rev=1355258&r1=1355257&r2=1355258&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleMarkFailure.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleMarkFailure.java Fri Jun 29 08:07:56 2012
@@ -47,11 +47,8 @@ public final class TestRuleMarkFailure i
         try {
           s.evaluate();
         } catch (Throwable t) {
-          for (Throwable t2 : expandFromMultiple(t)) {
-            if (!(t2 instanceof AssumptionViolatedException)) {
-              markFailed();
-              break;
-            }
+          if (!isAssumption(t)) {
+            markFailed();
           }
           throw t;
         }
@@ -60,6 +57,19 @@ public final class TestRuleMarkFailure i
   }
 
   /**
+   * Is a given exception (or a MultipleFailureException) an 
+   * {@link AssumptionViolatedException}?
+   */
+  public static boolean isAssumption(Throwable t) {
+    for (Throwable t2 : expandFromMultiple(t)) {
+      if (!(t2 instanceof AssumptionViolatedException)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
    * Expand from multi-exception wrappers.
    */
   private static List<Throwable> expandFromMultiple(Throwable t) {