LUCENE-5622: Fail tests if they print over the given limit of bytes to System.out or System.err

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1591222 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Dawid Weiss 2014-04-30 09:38:47 +00:00
commit e1f76c5ab1
26 changed files with 336 additions and 38 deletions

View File

@ -219,7 +219,7 @@ public class TestBugInSomething extends BaseTokenStreamTestCase {
@Override
public boolean incrementToken() throws IOException {
if (input.incrementToken()) {
System.out.println(input.getClass().getSimpleName() + "->" + this.reflectAsString(false));
if (VERBOSE) System.out.println(input.getClass().getSimpleName() + "->" + this.reflectAsString(false));
return true;
} else {
return false;
@ -229,19 +229,19 @@ public class TestBugInSomething extends BaseTokenStreamTestCase {
@Override
public void end() throws IOException {
super.end();
System.out.println(input.getClass().getSimpleName() + ".end()");
if (VERBOSE) System.out.println(input.getClass().getSimpleName() + ".end()");
}
@Override
public void close() throws IOException {
super.close();
System.out.println(input.getClass().getSimpleName() + ".close()");
if (VERBOSE) System.out.println(input.getClass().getSimpleName() + ".close()");
}
@Override
public void reset() throws IOException {
super.reset();
System.out.println(input.getClass().getSimpleName() + ".reset()");
if (VERBOSE) System.out.println(input.getClass().getSimpleName() + ".reset()");
}
}

View File

@ -70,7 +70,7 @@ import java.util.Locale;
import java.util.StringTokenizer;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
public class TestCompile extends LuceneTestCase {

View File

@ -31,6 +31,7 @@ import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;

View File

@ -19,6 +19,7 @@ package org.apache.lucene.analysis.uima;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;

View File

@ -26,7 +26,7 @@ import java.io.StringReader;
import org.apache.lucene.benchmark.byTask.Benchmark;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.junit.AfterClass;
import org.junit.BeforeClass;

View File

@ -62,6 +62,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
/**
* Test very simply that perf tasks - simple algorithms - are doing what they should.

View File

@ -37,9 +37,7 @@ import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import com.carrotsearch.randomizedtesting.RandomizedTest;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import conf.ConfLoader;
@ -52,8 +50,7 @@ public class TestPerfTasksParse extends LuceneTestCase {
// properties in effect in all tests here
static final String propPart =
INDENT + "directory=RAMDirectory" + NEW_LINE +
INDENT + "print.props=false" + NEW_LINE
;
INDENT + "print.props=false" + NEW_LINE;
/** Test the repetiotion parsing for parallel tasks */
public void testParseParallelTaskSequenceRepetition() throws Exception {

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import java.io.BufferedReader;
import java.io.File;

View File

@ -118,7 +118,7 @@
<property name="tests.verbose" value="false"/>
<property name="tests.infostream" value="${tests.verbose}"/>
<property name="tests.filterstacks" value="true"/>
<condition property="tests.heapsize" value="768M">
<isset property="run.clover"/>
</condition>

View File

@ -24,5 +24,4 @@ public class TestFastCompressionMode extends AbstractTestLZ4CompressionMode {
super.setUp();
mode = CompressionMode.FAST;
}
}

View File

@ -106,7 +106,7 @@ public class TestExceptionInBeforeClassHooks extends WithNestedTests {
@Test
public void testExceptionInBeforeClassFailsTheTest() {
Result runClasses = JUnitCore.runClasses(Nested1.class);
Assert.assertEquals(1, runClasses.getFailureCount());
assertFailureCount(1, runClasses);
Assert.assertEquals(1, runClasses.getRunCount());
Assert.assertTrue(runClasses.getFailures().get(0).getTrace().contains("foobar"));
}
@ -114,7 +114,7 @@ public class TestExceptionInBeforeClassHooks extends WithNestedTests {
@Test
public void testExceptionWithinTestFailsTheTest() {
Result runClasses = JUnitCore.runClasses(Nested2.class);
Assert.assertEquals(3, runClasses.getFailureCount());
assertFailureCount(3, runClasses);
Assert.assertEquals(3, runClasses.getRunCount());
ArrayList<String> foobars = new ArrayList<>();
@ -133,7 +133,7 @@ public class TestExceptionInBeforeClassHooks extends WithNestedTests {
@Test
public void testExceptionWithinBefore() {
Result runClasses = JUnitCore.runClasses(Nested3.class);
Assert.assertEquals(1, runClasses.getFailureCount());
assertFailureCount(1, runClasses);
Assert.assertEquals(1, runClasses.getRunCount());
Assert.assertTrue(runClasses.getFailures().get(0).getTrace().contains("foobar"));
}

View File

@ -22,7 +22,6 @@ import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.JUnitCore;
import org.junit.runner.Result;
import org.junit.runner.notification.Failure;
import com.carrotsearch.randomizedtesting.RandomizedTest;
@ -30,7 +29,7 @@ public class TestFailIfDirectoryNotClosed extends WithNestedTests {
public TestFailIfDirectoryNotClosed() {
super(true);
}
public static class Nested1 extends WithNestedTests.AbstractNestedTest {
public void testDummy() throws Exception {
Directory dir = newDirectory();
@ -43,11 +42,7 @@ public class TestFailIfDirectoryNotClosed extends WithNestedTests {
Result r = JUnitCore.runClasses(Nested1.class);
RandomizedTest.assumeTrue("Ignoring nested test, very likely zombie threads present.",
r.getIgnoreCount() == 0);
for (Failure f : r.getFailures()) {
System.out.println("Failure: " + f);
}
Assert.assertEquals(1, r.getFailureCount());
assertFailureCount(1, r);
Assert.assertTrue(r.getFailures().get(0).toString().contains("Resource in scope SUITE failed to close"));
}
}

View File

@ -3,6 +3,7 @@ package org.apache.lucene.util.junitcompat;
import java.util.*;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.junit.*;
import org.junit.rules.RuleChain;
import org.junit.rules.TestRule;

View File

@ -18,6 +18,7 @@ package org.apache.lucene.util.junitcompat;
*/
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.JUnitCore;
@ -53,7 +54,7 @@ public class TestSeedFromUncaught extends WithNestedTests {
@Test
public void testUncaughtDumpsSeed() {
Result result = JUnitCore.runClasses(ThrowInUncaught.class);
Assert.assertEquals(1, result.getFailureCount());
assertFailureCount(1, result);
Failure f = result.getFailures().get(0);
String trace = f.getTrace();
Assert.assertTrue(trace.contains("SeedInfo.seed("));

View File

@ -29,6 +29,7 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestRuleIgnoreAfterMaxFailures;
import org.apache.lucene.util.TestRuleIgnoreTestSuites;
import org.apache.lucene.util.TestRuleMarkFailure;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.junit.After;
import org.junit.Assert;
import org.junit.Assume;
@ -37,6 +38,8 @@ import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.rules.RuleChain;
import org.junit.rules.TestRule;
import org.junit.runner.Result;
import org.junit.runner.notification.Failure;
import com.carrotsearch.randomizedtesting.RandomizedRunner;
import com.carrotsearch.randomizedtesting.RandomizedTest;
@ -57,6 +60,7 @@ import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter;
* cause havoc (static fields).
*/
public abstract class WithNestedTests {
@SuppressSysoutChecks(bugUrl = "WithNestedTests has its own stream capture.")
public static abstract class AbstractNestedTest extends LuceneTestCase
implements TestRuleIgnoreTestSuites.NestedTestSuite {
protected static boolean isRunningNested() {
@ -164,6 +168,20 @@ public abstract class WithNestedTests {
}
}
protected void assertFailureCount(int expected, Result result) {
if (result.getFailureCount() != expected) {
StringBuilder b = new StringBuilder();
for (Failure f : result.getFailures()) {
b.append("\n\n");
b.append(f.getMessage());
b.append("\n");
b.append(f.getTrace());
}
RandomizedTest.assertFalse("Expected failures: " + expected + " but was " +
result.getFailureCount() + ", failures below: " + b.toString(), true);
}
}
protected String getSysOut() {
Assert.assertTrue(suppressOutputStreams);
System.out.flush();

View File

@ -23,7 +23,7 @@ import java.io.PrintStream;
import java.nio.charset.Charset;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
public class TestDemo extends LuceneTestCase {

View File

@ -133,7 +133,6 @@ public class TestDrillDownQuery extends FacetTestCase {
// Making sure the query yields 25 documents with the facet "a"
DrillDownQuery q = new DrillDownQuery(config);
q.add("a");
System.out.println("q=" + q);
QueryUtils.check(q);
TopDocs docs = searcher.search(q, 100);
assertEquals(25, docs.totalHits);

View File

@ -17,9 +17,6 @@
package org.apache.lucene.document;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import java.util.Set;
import java.util.HashSet;
import java.util.Map;
@ -28,7 +25,6 @@ import java.io.IOException;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.store.*;
import org.apache.lucene.document.*;
import org.apache.lucene.analysis.*;
import org.apache.lucene.index.*;
import org.apache.lucene.search.*;
@ -118,7 +114,7 @@ public class TestLazyDocument extends LuceneTestCase {
assertFalse(f.name() + " is loaded", lf.hasBeenLoaded());
}
}
System.out.println("numFieldValues == " + numFieldValues);
if (VERBOSE) System.out.println("numFieldValues == " + numFieldValues);
assertEquals("numFieldValues", 1 + (NUM_VALUES * FIELDS.length),
numFieldValues);

View File

@ -22,6 +22,7 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
public class TestMultiPassIndexSplitter extends LuceneTestCase {
IndexReader input;

View File

@ -40,10 +40,18 @@ import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.servlet.ServletHandler;
import org.eclipse.jetty.servlet.ServletHolder;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.RuleChain;
import org.junit.rules.TestRule;
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
public class HttpReplicatorTest extends ReplicatorTestCase {
@Rule
public TestRule testRules =
RuleChain.outerRule(new SystemPropertiesRestoreRule());
private File clientWorkDir;
private Replicator serverReplicator;
private IndexWriter writer;
@ -69,7 +77,9 @@ public class HttpReplicatorTest extends ReplicatorTestCase {
@Override
public void setUp() throws Exception {
super.setUp();
System.setProperty("org.eclipse.jetty.LEVEL", "DEBUG"); // sets stderr logging to DEBUG level
if (VERBOSE) {
System.setProperty("org.eclipse.jetty.LEVEL", "DEBUG"); // sets stderr logging to DEBUG level
}
clientWorkDir = createTempDir("httpReplicatorTest");
handlerIndexDir = newDirectory();
serverIndexDir = newDirectory();
@ -87,7 +97,6 @@ public class HttpReplicatorTest extends ReplicatorTestCase {
stopHttpServer(server);
writer.rollback();
IOUtils.close(reader, handlerIndexDir, serverIndexDir);
System.clearProperty("org.eclipse.jetty.LEVEL");
super.tearDown();
}

View File

@ -35,6 +35,8 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.apache.lucene.util.TestRuleLimitSysouts.Limit;
import org.junit.After;
import org.junit.Before;
@ -47,6 +49,7 @@ import static com.carrotsearch.randomizedtesting.RandomizedTest.randomGaussian;
import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
/** A base test class for spatial lucene. It's mostly Lucene generic. */
@SuppressSysoutChecks(bugUrl = "These tests use JUL extensively.")
public abstract class SpatialTestCase extends LuceneTestCase {
private DirectoryReader indexReader;

View File

@ -230,6 +230,7 @@ import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAs
@ThreadLeakFilters(defaultFilters = true, filters = {
QuickPatchThreadsFilter.class
})
@TestRuleLimitSysouts.Limit(bytes = TestRuleLimitSysouts.DEFAULT_SYSOUT_BYTES_THRESHOLD)
public abstract class LuceneTestCase extends Assert {
// --------------------------------------------------------------------
@ -348,6 +349,21 @@ public abstract class LuceneTestCase extends Assert {
public String bugUrl() default "None";
}
/**
* Ignore {@link TestRuleLimitSysouts} for any suite which is known to print
* over the default limit of bytes to {@link System#out} or {@link System#err}.
*
* @see TestRuleLimitSysouts
*/
@Documented
@Inherited
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.TYPE)
public @interface SuppressSysoutChecks {
/** Point to JIRA entry. */
public String bugUrl();
}
// -----------------------------------------------------------------
// Truly immutable fields and constants, initialized once and valid
// for all suites ever since.
@ -362,11 +378,13 @@ public abstract class LuceneTestCase extends Assert {
/**
* True if and only if tests are run in verbose mode. If this flag is false
* tests are not expected to print any messages.
* tests are not expected to print any messages. Enforced with {@link TestRuleLimitSysouts}.
*/
public static final boolean VERBOSE = systemPropertyAsBoolean("tests.verbose", false);
/** TODO: javadoc? */
/**
* Enables or disables dumping of {@link InfoStream} messages.
*/
public static final boolean INFOSTREAM = systemPropertyAsBoolean("tests.infostream", VERBOSE);
/**
@ -480,7 +498,7 @@ public abstract class LuceneTestCase extends Assert {
/**
* Suite failure marker (any error in the test or suite scope).
*/
public static TestRuleMarkFailure suiteFailureMarker;
private static TestRuleMarkFailure suiteFailureMarker;
/**
* Ignore tests after hitting a designated number of initial failures. This
@ -513,6 +531,15 @@ public abstract class LuceneTestCase extends Assert {
new TestRuleIgnoreAfterMaxFailures(maxFailures));
ignoreAfterMaxFailures = TestRuleDelegate.of(ignoreAfterMaxFailuresDelegate);
}
/**
* Try to capture streams early so that other classes don't have a chance to steal references
* to them (as is the case with ju.logging handlers).
*/
static {
TestRuleLimitSysouts.checkCaptureStreams();
Logger.getGlobal().getHandlers();
}
/**
* Temporarily substitute the global {@link TestRuleIgnoreAfterMaxFailures}. See
@ -547,6 +574,7 @@ public abstract class LuceneTestCase extends Assert {
.around(ignoreAfterMaxFailures)
.around(suiteFailureMarker = new TestRuleMarkFailure())
.around(new TestRuleAssertionsRequired())
.around(new TestRuleLimitSysouts(suiteFailureMarker))
.around(new TemporaryFilesCleanupRule())
.around(new StaticFieldsInvariantRule(STATIC_LEAK_THRESHOLD, true) {
@Override
@ -2406,6 +2434,12 @@ public abstract class LuceneTestCase extends Assert {
}
}
/**
* Checks and cleans up temporary files.
*
* @see LuceneTestCase#createTempDir()
* @see LuceneTestCase#createTempFile()
*/
private static class TemporaryFilesCleanupRule extends TestRuleAdapter {
@Override
protected void before() throws Throwable {

View File

@ -0,0 +1,236 @@
package org.apache.lucene.util;
import java.io.FilterOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
import java.lang.annotation.Documented;
import java.lang.annotation.ElementType;
import java.lang.annotation.Inherited;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import java.nio.charset.Charset;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import com.carrotsearch.randomizedtesting.RandomizedTest;
import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter;
/*
* 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.
*/
/**
* Fails the suite if it prints over the given limit of bytes to either
* {@link System#out} or {@link System#err},
* unless the condition is not enforced (see {@link #isEnforced()}).
*/
public class TestRuleLimitSysouts extends TestRuleAdapter {
/**
* Max limit of bytes printed to either {@link System#out} or {@link System#err}.
* This limit is enforced per-class (suite).
*/
public final static int DEFAULT_SYSOUT_BYTES_THRESHOLD = 8 * 1024;
/**
* An annotation specifying the limit of bytes per class.
*/
@Documented
@Inherited
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.TYPE)
public static @interface Limit {
public int bytes();
}
private final static AtomicInteger bytesWritten = new AtomicInteger();
private final static DelegateStream capturedSystemOut;
private final static DelegateStream capturedSystemErr;
/**
* We capture system output and error streams as early as possible because
* certain components (like the Java logging system) steal these references and
* never refresh them.
*
* Also, for this exact reason, we cannot change delegate streams for every suite.
* This isn't as elegant as it should be, but there's no workaround for this.
*/
static {
System.out.flush();
System.err.flush();
final String csn = Charset.defaultCharset().name();
capturedSystemOut = new DelegateStream(System.out, csn, bytesWritten);
capturedSystemErr = new DelegateStream(System.err, csn, bytesWritten);
System.setOut(capturedSystemOut.printStream);
System.setErr(capturedSystemErr.printStream);
}
/**
* Test failures from any tests or rules before.
*/
private final TestRuleMarkFailure failureMarker;
/**
* Tracks the number of bytes written to an underlying stream by
* incrementing an {@link AtomicInteger}.
*/
static class DelegateStream extends FilterOutputStream {
final PrintStream printStream;
final AtomicInteger bytesCounter;
public DelegateStream(OutputStream delegate, String charset, AtomicInteger bytesCounter) {
super(delegate);
try {
this.printStream = new PrintStream(this, true, charset);
this.bytesCounter = bytesCounter;
} catch (UnsupportedEncodingException e) {
throw new RuntimeException(e);
}
}
// Do override all three write() methods to make sure nothing slips through.
@Override
public void write(byte[] b) throws IOException {
if (b.length > 0) {
bytesCounter.addAndGet(b.length);
}
super.write(b);
}
@Override
public void write(byte[] b, int off, int len) throws IOException {
if (len > 0) {
bytesCounter.addAndGet(len);
}
super.write(b, off, len);
}
@Override
public void write(int b) throws IOException {
bytesCounter.incrementAndGet();
super.write(b);
}
}
public TestRuleLimitSysouts(TestRuleMarkFailure failureMarker) {
this.failureMarker = failureMarker;
}
/** */
@Override
protected void before() throws Throwable {
if (isEnforced()) {
checkCaptureStreams();
}
resetCaptureState();
validateClassAnnotations();
}
private void validateClassAnnotations() {
Class<?> target = RandomizedTest.getContext().getTargetClass();
if (target.isAnnotationPresent(Limit.class)) {
int bytes = target.getAnnotation(Limit.class).bytes();
if (bytes < 0 || bytes > 1 * 1024 * 1024) {
throw new AssertionError("The sysout limit is insane. Did you want to use "
+ "@" + LuceneTestCase.SuppressSysoutChecks.class.getName() + " annotation to "
+ "avoid sysout checks entirely?");
}
}
}
/**
* Ensures {@link System#out} and {@link System#err} point to delegate streams.
*/
public static void checkCaptureStreams() {
// Make sure we still hold the right references to wrapper streams.
if (System.out != capturedSystemOut.printStream) {
throw new AssertionError("Something has changed System.out to: " + System.out.getClass().getName());
}
if (System.err != capturedSystemErr.printStream) {
throw new AssertionError("Something has changed System.err to: " + System.err.getClass().getName());
}
}
protected boolean isEnforced() {
Class<?> target = RandomizedTest.getContext().getTargetClass();
if (LuceneTestCase.VERBOSE ||
LuceneTestCase.INFOSTREAM ||
target.isAnnotationPresent(SuppressSysoutChecks.class)) {
return false;
}
if (!target.isAnnotationPresent(Limit.class)) {
return false;
}
return true;
}
/**
* We're only interested in failing the suite if it was successful. Otherwise
* just propagate the original problem and don't bother.
*/
@Override
protected void afterIfSuccessful() throws Throwable {
if (isEnforced()) {
checkCaptureStreams();
// Flush any buffers.
capturedSystemOut.printStream.flush();
capturedSystemErr.printStream.flush();
// Check for offenders, but only if everything was successful so far.
int limit = RandomizedTest.getContext().getTargetClass().getAnnotation(Limit.class).bytes();
if (bytesWritten.get() >= limit && failureMarker.wasSuccessful()) {
throw new AssertionError(String.format(Locale.ENGLISH,
"The test or suite printed %d bytes to stdout and stderr," +
" even though the limit was set to %d bytes. Increase the limit with @%s, ignore it completely" +
" with @%s or run with -Dtests.verbose=true",
bytesWritten.get(),
limit,
Limit.class.getSimpleName(),
SuppressSysoutChecks.class.getSimpleName()));
}
}
}
/**
* Restore original streams.
*/
@Override
protected void afterAlways(List<Throwable> errors) throws Throwable {
resetCaptureState();
}
private void resetCaptureState() {
capturedSystemOut.printStream.flush();
capturedSystemErr.printStream.flush();
bytesWritten.set(0);
}
}

View File

@ -24,6 +24,7 @@ import java.util.List;
import java.util.Map;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@ -58,6 +59,7 @@ import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
* MiniSolrCloudCluster is designed to be used outside of the Lucene test
* hierarchy.
*/
@SuppressSysoutChecks(bugUrl = "Solr logs to JUL")
public class TestMiniSolrCloudCluster extends LuceneTestCase {
private static Logger log = LoggerFactory.getLogger(MiniSolrCloudCluster.class);

View File

@ -19,6 +19,7 @@ package org.apache.solr.util;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
@ -28,8 +29,8 @@ import java.lang.reflect.InvocationTargetException;
* A "test the test" sanity check using reflection to ensure that
* {@linke RandomMergePolicy} is working as expected
*/
@SuppressSysoutChecks(bugUrl = "Logs to JUL")
public class TestRandomMergePolicy extends LuceneTestCase {
/**
* Ensure every MP method is overridden by RMP
* (future proof ourselves against new methods being added to MP)

View File

@ -58,6 +58,7 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.QuickPatchThreadsFilter;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.client.solrj.util.ClientUtils;
@ -118,6 +119,7 @@ import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
SolrIgnoredThreadsFilter.class,
QuickPatchThreadsFilter.class
})
@SuppressSysoutChecks(bugUrl = "Solr dumps tons of logs to console.")
public abstract class SolrTestCaseJ4 extends LuceneTestCase {
private static String coreName = ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME;
public static int DEFAULT_CONNECTION_TIMEOUT = 60000; // default socket connection timeout in ms