summaryrefslogtreecommitdiffstats
path: root/vespa-osgi-testrunner
diff options
context:
space:
mode:
authorJon Marius Venstad <jonmv@users.noreply.github.com>2022-06-07 11:38:14 +0200
committerGitHub <noreply@github.com>2022-06-07 11:38:14 +0200
commit36663583469ce63be5467907dd8e976d798f8f31 (patch)
tree80b2107efe63554f55a380462df86f9d4671217c /vespa-osgi-testrunner
parent0fd0880cfcfa5a1064ee78a63fc9f20d5a03183b (diff)
parentb760d66aefe51e44875673b0035e1ed4cedaaa2d (diff)
Merge pull request #22928 from vespa-engine/jonmv/junit-test-runner-2
Reimplement JUnit integratino, with unit tests, and structured report
Diffstat (limited to 'vespa-osgi-testrunner')
-rw-r--r--vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/AggregateTestRunner.java18
-rw-r--r--vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/JunitRunner.java95
-rw-r--r--vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TeeStream.java65
-rw-r--r--vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReport.java358
-rw-r--r--vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReportGeneratingListener.java183
-rw-r--r--vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestRunnerHandler.java103
-rw-r--r--vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/VespaJunitLogListener.java150
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/DisabledClassTest.java17
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/DisabledTest.java14
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAfterAllTest.java19
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAfterEachTest.java16
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAssertionTest.java13
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAssumptionTest.java14
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllAssertionTest.java20
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllTest.java19
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllTestFactoryTest.java26
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeEachTest.java16
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingClassAssumptionTest.java19
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingClassLoadingTest.java19
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingExtensionTest.java23
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInnerClassTest.java28
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationAssertionTest.java19
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationTest.java7
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTest.java12
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTestAndBothAftersTest.java21
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTestFactoryTest.java26
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/InconclusiveTest.java15
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/NotInconclusiveTest.java13
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/SampleTest.java119
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/SucceedingTest.java13
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/TimingOutTest.java18
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/UsingTestRuntimeTest.java22
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/WrongBeforeAllTest.java19
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/AggregateTestRunnerTest.java46
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/Expect.java29
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/JunitRunnerTest.java117
-rw-r--r--vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/TestRunnerHandlerTest.java85
-rw-r--r--vespa-osgi-testrunner/src/test/resources/output.json82
-rw-r--r--vespa-osgi-testrunner/src/test/resources/report.json430
39 files changed, 1950 insertions, 378 deletions
diff --git a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/AggregateTestRunner.java b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/AggregateTestRunner.java
index d00184f8b04..7b2bb26b444 100644
--- a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/AggregateTestRunner.java
+++ b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/AggregateTestRunner.java
@@ -100,23 +100,7 @@ public class AggregateTestRunner implements TestRunner {
}
static TestReport merge(TestReport first, TestReport second) {
- return first == null ? second
- : second == null ? first
- : TestReport.builder()
- .withAbortedCount(first.abortedCount + second.abortedCount)
- .withFailedCount(first.failedCount + second.failedCount)
- .withIgnoredCount(first.ignoredCount + second.ignoredCount)
- .withSuccessCount(first.successCount + second.successCount)
- .withFailures(merged(first.failures, second.failures))
- .withLogs(merged(first.logLines, second.logLines))
- .build();
- }
-
- static <T> List<T> merged(List<T> first, List<T> second) {
- ArrayList<T> merged = new ArrayList<>();
- merged.addAll(first);
- merged.addAll(second);
- return merged;
+ return first == null ? second : second == null ? first : first.mergedWith(second);
}
}
diff --git a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/JunitRunner.java b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/JunitRunner.java
index c01c9b571e0..54f0941208d 100644
--- a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/JunitRunner.java
+++ b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/JunitRunner.java
@@ -10,6 +10,9 @@ import com.yahoo.component.AbstractComponent;
import com.yahoo.component.annotation.Inject;
import com.yahoo.jdisc.application.OsgiFramework;
import com.yahoo.vespa.defaults.Defaults;
+import com.yahoo.vespa.testrunner.TestReport.ContainerNode;
+import com.yahoo.vespa.testrunner.TestReport.FailureNode;
+import com.yahoo.vespa.testrunner.TestReport.Status;
import org.junit.jupiter.engine.JupiterTestEngine;
import org.junit.platform.engine.discovery.DiscoverySelectors;
import org.junit.platform.launcher.LauncherDiscoveryRequest;
@@ -19,8 +22,10 @@ import org.junit.platform.launcher.core.LauncherDiscoveryRequestBuilder;
import org.junit.platform.launcher.core.LauncherFactory;
import org.junit.platform.launcher.listeners.SummaryGeneratingListener;
+import java.time.Clock;
import java.util.Collection;
import java.util.List;
+import java.util.Map;
import java.util.Optional;
import java.util.SortedMap;
import java.util.concurrent.CompletableFuture;
@@ -36,12 +41,16 @@ import static java.util.stream.Collectors.toList;
/**
* @author mortent
+ * @author jonmv
*/
public class JunitRunner extends AbstractComponent implements TestRunner {
private static final Logger logger = Logger.getLogger(JunitRunner.class.getName());
+ private final Clock clock;
private final SortedMap<Long, LogRecord> logRecords = new ConcurrentSkipListMap<>();
+ private final TeeStream stdoutTee = TeeStream.ofSystemOut();
+ private final TeeStream stderrTee = TeeStream.ofSystemErr();
private final TestRuntimeProvider testRuntimeProvider;
private final Function<Suite, List<Class<?>>> classLoader;
private final BiConsumer<LauncherDiscoveryRequest, TestExecutionListener[]> testExecutor;
@@ -52,18 +61,22 @@ public class JunitRunner extends AbstractComponent implements TestRunner {
JunitTestRunnerConfig config,
TestRuntimeProvider testRuntimeProvider,
SystemInfo systemInfo) {
- this(testRuntimeProvider,
+ this(Clock.systemUTC(),
+ testRuntimeProvider,
new TestBundleLoader(osgiFramework)::loadTestClasses,
(discoveryRequest, listeners) -> LauncherFactory.create(LauncherConfig.builder()
.addTestEngines(new JupiterTestEngine())
.build()).execute(discoveryRequest, listeners));
uglyHackSetCredentialsRootSystemProperty(config, systemInfo.zone());
+
}
- JunitRunner(TestRuntimeProvider testRuntimeProvider,
- Function<Suite, List<Class<?>>> classLoader,
- BiConsumer<LauncherDiscoveryRequest, TestExecutionListener[]> testExecutor) {
+ JunitRunner(Clock clock,
+ TestRuntimeProvider testRuntimeProvider,
+ Function<Suite, List<Class<?>>> classLoader,
+ BiConsumer<LauncherDiscoveryRequest, TestExecutionListener[]> testExecutor) {
+ this.clock = clock;
this.classLoader = classLoader;
this.testExecutor = testExecutor;
this.testRuntimeProvider = testRuntimeProvider;
@@ -76,10 +89,9 @@ public class JunitRunner extends AbstractComponent implements TestRunner {
}
try {
logRecords.clear();
- testRuntimeProvider.initialize(testConfig);
- execution = CompletableFuture.supplyAsync(() -> launchJunit(suite));
- } catch (Exception e) {
- execution = CompletableFuture.completedFuture(createReportWithFailedInitialization(e));
+ execution = CompletableFuture.supplyAsync(() -> launchJunit(suite, testConfig));
+ } catch (Throwable t) {
+ execution = CompletableFuture.completedFuture(TestReport.createFailed(clock, suite, t));
}
return execution;
}
@@ -89,52 +101,25 @@ public class JunitRunner extends AbstractComponent implements TestRunner {
return logRecords.tailMap(after + 1).values();
}
- static TestReport createReportWithFailedInitialization(Exception exception) {
- TestReport.Failure failure = new TestReport.Failure("init", exception);
- return new TestReport.Builder().withFailures(List.of(failure))
- .withFailedCount(1)
- .build();
- }
-
-
- private TestReport launchJunit(Suite suite) {
+ private TestReport launchJunit(Suite suite, byte[] testConfig) {
List<Class<?>> testClasses = classLoader.apply(suite);
if (testClasses == null)
return null;
- VespaJunitLogListener logListener = new VespaJunitLogListener(record -> logRecords.put(record.getSequenceNumber(), record));
- SummaryGeneratingListener summaryListener = new SummaryGeneratingListener();
+ testRuntimeProvider.initialize(testConfig);
+ TestReportGeneratingListener testReportListener = new TestReportGeneratingListener(suite,
+ record -> logRecords.put(record.getSequenceNumber(), record),
+ stdoutTee,
+ stderrTee,
+ clock);
LauncherDiscoveryRequest discoveryRequest = LauncherDiscoveryRequestBuilder.request()
.selectors(testClasses.stream()
.map(DiscoverySelectors::selectClass)
.collect(toList()))
.build();
+ testExecutor.accept(discoveryRequest, new TestExecutionListener[] { testReportListener });
- testExecutor.accept(discoveryRequest, new TestExecutionListener[] { logListener, summaryListener });
-
- var report = summaryListener.getSummary();
- var failures = report.getFailures().stream()
- .map(failure -> {
- TestReport.trimStackTraces(failure.getException(), JunitRunner.class.getName());
- return new TestReport.Failure(VespaJunitLogListener.toString(failure.getTestIdentifier().getUniqueIdObject()),
- failure.getException());
- })
- .collect(toList());
-
- // TODO: move to aggregator.
- long inconclusive = suite == Suite.PRODUCTION_TEST ? failures.stream()
- .filter(failure -> failure.exception() instanceof InconclusiveTestException)
- .count()
- : 0;
- return TestReport.builder()
- .withSuccessCount(report.getTestsSucceededCount())
- .withAbortedCount(report.getTestsAbortedCount())
- .withIgnoredCount(report.getTestsSkippedCount())
- .withFailedCount(report.getTestsFailedCount() - inconclusive)
- .withInconclusiveCount(inconclusive)
- .withFailures(failures)
- .withLogs(logRecords.values())
- .build();
+ return testReportListener.report();
}
@Override
@@ -147,23 +132,37 @@ public class JunitRunner extends AbstractComponent implements TestRunner {
if (execution == null) return TestRunner.Status.NOT_STARTED;
if ( ! execution.isDone()) return TestRunner.Status.RUNNING;
try {
- return execution.get() == null ? Status.NO_TESTS : execution.get().status();
+ return testRunnerStatus(execution.get());
} catch (InterruptedException | ExecutionException e) {
logger.log(Level.WARNING, "Error while getting test report", e);
return TestRunner.Status.ERROR;
}
}
+ static TestRunner.Status testRunnerStatus(TestReport report) {
+ if (report == null) return Status.NO_TESTS;
+ switch (report.root().status()) {
+ case error:
+ case failed: return Status.FAILURE;
+ case inconclusive: return Status.INCONCLUSIVE;
+ case successful:
+ case skipped:
+ case aborted: return report.root().tally().containsKey(TestReport.Status.successful) ? Status.SUCCESS
+ : Status.NO_TESTS;
+ default: throw new IllegalStateException("unknown status '" + report.root().status() + "'");
+ }
+ }
+
@Override
public TestReport getReport() {
if (execution.isDone()) {
try {
return execution.get();
- } catch (Exception e) {
- logger.log(Level.WARNING, "Error getting test report", e);
+ } catch (Throwable t) {
+ logger.log(Level.WARNING, "Error getting test report", t);
// Likely this is something wrong with the provided test bundle. Create a test report
// and present in the console to enable tenants to act on it.
- return createReportWithFailedInitialization(e);
+ return TestReport.createFailed(clock, null, t);
}
} else {
return null;
diff --git a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TeeStream.java b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TeeStream.java
new file mode 100644
index 00000000000..bef4c8de1b6
--- /dev/null
+++ b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TeeStream.java
@@ -0,0 +1,65 @@
+package com.yahoo.vespa.testrunner;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Used to replace System.out and System.err, providing the ability to forward output to an additional sink.
+ *
+ * @author jonmv
+ */
+public class TeeStream extends OutputStream {
+
+ private final AtomicReference<OutputStream> tee = new AtomicReference<>();
+ private final OutputStream original;
+
+ private TeeStream(OutputStream original) {
+ this.original = original;
+ }
+
+ public static TeeStream ofSystemOut() {
+ TeeStream teed = new TeeStream(System.out);
+ System.setOut(new PrintStream(teed));
+ return teed;
+ }
+
+ public static TeeStream ofSystemErr() {
+ TeeStream teed = new TeeStream(System.err);
+ System.setErr(new PrintStream(teed));
+ return teed;
+ }
+
+ public void setTee(OutputStream tee) {
+ if ( ! this.tee.compareAndSet(null, tee)) throw new IllegalStateException("tee already set");
+ }
+
+ public OutputStream clearTee() {
+ OutputStream tee = this.tee.getAndSet(null);
+ if (tee == null) throw new IllegalStateException("tee not set");
+ return tee;
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ OutputStream maybe = tee.get();
+ if (maybe != null) maybe.write(b);
+ original.write(b);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ OutputStream maybe = tee.get();
+ if (maybe != null) maybe.write(b, off, len);
+ original.write(b, off, len);
+ }
+
+ @Override
+ public void flush() throws IOException {
+ OutputStream maybe = tee.get();
+ if (maybe != null) maybe.flush();
+ original.flush();
+ }
+
+}
diff --git a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReport.java b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReport.java
index 747005f467d..9aae329d7fb 100644
--- a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReport.java
+++ b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReport.java
@@ -1,125 +1,312 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.testrunner;
-import java.util.Collection;
-import java.util.Collections;
+import ai.vespa.hosted.cd.InconclusiveTestException;
+import com.yahoo.collections.Comparables;
+import com.yahoo.vespa.testrunner.TestRunner.Suite;
+import org.junit.platform.engine.UniqueId;
+import org.junit.platform.engine.UniqueId.Segment;
+import org.junit.platform.launcher.TestIdentifier;
+import org.junit.platform.launcher.TestPlan;
+
+import java.time.Clock;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.EnumMap;
+import java.util.HashSet;
import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
import java.util.logging.LogRecord;
-import static com.yahoo.vespa.testrunner.TestRunner.Status.FAILURE;
-import static com.yahoo.vespa.testrunner.TestRunner.Status.INCONCLUSIVE;
-import static com.yahoo.vespa.testrunner.TestRunner.Status.NO_TESTS;
-import static com.yahoo.vespa.testrunner.TestRunner.Status.SUCCESS;
import static java.util.Arrays.copyOf;
/**
- * @author mortent
+ * @author jonmv
*/
public class TestReport {
- final long successCount;
- final long failedCount;
- final long inconclusiveCount;
- final long ignoredCount;
- final long abortedCount;
- final List<Failure> failures;
- final List<LogRecord> logLines;
+ private final Object monitor = new Object();
+ private final Set<TestIdentifier> complete = new HashSet<>();
+ private final Clock clock;
+ private final ContainerNode root;
+ private final Suite suite;
+ private NamedNode current;
+ private TestPlan plan;
+
+ private TestReport(Clock clock, Suite suite, ContainerNode root) {
+ this.clock = clock;
+ this.root = root;
+ this.current = root;
+ this.suite = suite;
+ }
+
+ TestReport(Clock clock, Suite suite) {
+ this(clock, suite, new ContainerNode(null, null, toString(suite), clock.instant()));
+ }
+
+ static TestReport createFailed(Clock clock, Suite suite, Throwable thrown) {
+ if (thrown instanceof OutOfMemoryError) throw (Error) thrown;
+ TestReport failed = new TestReport(clock, suite);
+ failed.complete();
+ failed.root().children.add(new FailureNode(failed.root(), thrown, suite));
+ return failed;
+ }
+
+ /** Verify the path from the root to the current node corresponds to the given id. */
+ private void verifyStructure(NamedNode node, UniqueId id) {
+ Deque<String> path = new ArrayDeque<>();
+ while (node != root) {
+ path.push(node.id);
+ node = node.parent;
+ }
+ Deque<String> segments = new ArrayDeque<>();
+ if (id != null) for (Segment segment : id.getSegments())
+ segments.add(segment.getValue());
+
+ if ( ! List.copyOf(path).equals(List.copyOf(segments)))
+ throw new IllegalStateException("test node " + segments + " referenced, but expected " + path);
+ }
+
+ void start(TestPlan plan) {
+ synchronized (monitor) {
+ this.plan = plan;
+ }
+ }
+
+ void start(TestIdentifier id) {
+ synchronized (monitor) {
+ NamedNode child = id.isTest() ? new TestNode(current, id.getUniqueIdObject().getLastSegment().getValue(), id.getDisplayName(), clock.instant())
+ : new ContainerNode(current, id.getUniqueIdObject().getLastSegment().getValue(), id.getDisplayName(), clock.instant());
+ verifyStructure(child, id.getUniqueIdObject());
+ current.children.add(child);
+ current = child;
+ }
+ }
+
+ ContainerNode complete() {
+ synchronized (monitor) {
+ complete(null);
+ return root();
+ }
+ }
+
+ private NamedNode complete(TestIdentifier id) {
+ verifyStructure(current, id == null ? null : id.getUniqueIdObject());
+
+ Set<TestIdentifier> incomplete = id != null ? plan.getChildren(id) : plan != null ? plan.getRoots() : Set.of();
+ for (TestIdentifier child : incomplete) if ( ! complete.contains(child)) skip(child);
+ complete.add(id);
- private TestReport(long successCount, long failedCount, long inconclusiveCount, long ignoredCount, long abortedCount, List<Failure> failures, List<LogRecord> logLines) {
- this.successCount = successCount;
- this.failedCount = failedCount;
- this.inconclusiveCount = inconclusiveCount;
- this.ignoredCount = ignoredCount;
- this.abortedCount = abortedCount;
- this.failures = failures;
- this.logLines = logLines;
+ current.end = clock.instant();
+ NamedNode node = current;
+ current = current.parent;
+ return node;
}
- public List<LogRecord> logLines() {
- return logLines;
+ NamedNode skip(TestIdentifier id) {
+ synchronized (monitor) {
+ start(id);
+ current.status = Status.skipped;
+ return complete(id);
+ }
}
- public TestRunner.Status status() {
- return (failures.size() > 0 || failedCount > 0) ? FAILURE : inconclusiveCount > 0 ? INCONCLUSIVE : successCount > 0 ? SUCCESS : NO_TESTS;
+ NamedNode abort(TestIdentifier id) {
+ synchronized (monitor) {
+ current.status = Status.aborted;
+ return complete(id);
+ }
}
- public static Builder builder(){
- return new Builder();
+ NamedNode complete(TestIdentifier id, Throwable thrown) {
+ synchronized (monitor) {
+ Status status = Status.successful;
+ if (thrown != null) {
+ FailureNode failure = new FailureNode(current, thrown, suite);
+ current.children.add(failure);
+ status = failure.status();
+ }
+ current.status = status;
+ return complete(id);
+ }
}
+ void log(LogRecord record) {
+ synchronized (monitor) {
+ if (record.getThrown() != null) trimStackTraces(record.getThrown(), JunitRunner.class.getName());
+ if ( ! (current.children.peekLast() instanceof OutputNode))
+ current.children.add(new OutputNode(current));
+
+ ((OutputNode) current.children.peekLast()).log.add(record);
+ }
+ }
+
+ public TestReport mergedWith(TestReport other) {
+ synchronized (monitor) {
+ synchronized (other.monitor) {
+ if (current != null || other.current != null)
+ throw new IllegalArgumentException("can only merge completed test reports");
+
+ if (root.start().isAfter(other.root.start()))
+ throw new IllegalArgumentException("appended test report cannot have started before the one appended to");
+
+ ContainerNode newRoot = new ContainerNode(null, null, root.name(), root.start());
+ newRoot.children.addAll(root.children);
+ newRoot.children.addAll(other.root.children);
+ TestReport merged = new TestReport(clock, suite, newRoot);
+ merged.complete();
+ return merged;
+ }
+ }
+ }
+
+ public ContainerNode root() {
+ synchronized (monitor) {
+ return root;
+ }
+ }
+
+ public static class Node {
+
+ final Deque<Node> children = new ArrayDeque<>();
+ final NamedNode parent;
- public static class Builder {
+ Node(NamedNode parent) {
+ this.parent = parent;
+ }
- private long successCount;
- private long failedCount;
- private long inconclusiveCount;
- private long ignoredCount;
- private long abortedCount;
- private List<Failure> failures = Collections.emptyList();
- private List<LogRecord> logLines = Collections.emptyList();
+ Status status() {
+ int status = 0;
+ for (Node node : children)
+ status = Math.max(status, node.status().ordinal());
- public TestReport build() {
- return new TestReport(successCount, failedCount, inconclusiveCount, ignoredCount, abortedCount, failures, logLines);
+ return Status.values()[status];
}
- public Builder withSuccessCount(long successCount) {
- this.successCount = successCount;
- return this;
+ Map<Status, Long> tally() {
+ Map<Status, Long> tally = new EnumMap<>(Status.class);
+ for (Node child : children)
+ child.tally().forEach((status, count) -> tally.merge(status, count, Long::sum));
+
+ return tally;
}
- public Builder withFailedCount(long failedCount) {
- this.failedCount = failedCount;
- return this;
+ public Queue<Node> children() {
+ return children;
}
- public Builder withInconclusiveCount(long inconclusiveCount) {
- this.inconclusiveCount = inconclusiveCount;
- return this;
+ }
+
+ static abstract class NamedNode extends Node {
+
+ private final String id;
+ private final String name;
+ private final Instant start;
+ private Status status;
+ private Instant end;
+
+ NamedNode(NamedNode parent, String id, String name, Instant now) {
+ super(parent);
+ this.id = id;
+ this.name = name;
+ this.start = now;
+ }
+
+ @Override
+ public Status status() {
+ Status aggregate = super.status();
+ return status == null ? aggregate : Comparables.max(status, aggregate);
}
- public Builder withIgnoredCount(long ignoredCount) {
- this.ignoredCount = ignoredCount;
- return this;
+ public String name() {
+ return name;
}
- public Builder withAbortedCount(long abortedCount) {
- this.abortedCount = abortedCount;
- return this;
+ public Instant start() {
+ return start;
}
- public Builder withFailures(List<Failure> failures) {
- this.failures = List.copyOf(failures);
- return this;
+ public Duration duration() {
+ return Duration.between(start, end);
+ }
+
+ }
+
+ public static class ContainerNode extends NamedNode {
+
+ ContainerNode(NamedNode parent, String name, String display, Instant now) {
+ super(parent, name, display, now);
+ }
+
+ }
+
+ public static class TestNode extends NamedNode {
+
+ TestNode(NamedNode parent, String name, String display, Instant now) {
+ super(parent, name, display, now);
}
- public Builder withLogs(Collection<LogRecord> logRecords) {
- this.logLines = List.copyOf(logRecords);
- return this;
+ @Override
+ public Map<Status, Long> tally() {
+ return Map.of(status(), 1L);
}
}
+ public static class OutputNode extends Node {
- public static class Failure {
+ private final ArrayDeque<LogRecord> log = new ArrayDeque<>();
- private final String testId;
- private final Throwable exception;
+ public OutputNode(NamedNode parent) {
+ super(parent);
+ }
- public Failure(String testId, Throwable exception) {
- this.testId = testId;
- this.exception = exception;
+ public Queue<LogRecord> log() {
+ return log;
}
- public String testId() {
- return testId;
+ }
+
+ public static class FailureNode extends Node {
+
+ private final Throwable thrown;
+ private final Suite suite;
+
+ public FailureNode(NamedNode parent, Throwable thrown, Suite suite) {
+ super(parent);
+ this.thrown = thrown;
+ trimStackTraces(thrown, JunitRunner.class.getName());
+ this.suite = suite;
+ }
+
+ public Throwable thrown() {
+ return thrown;
}
- public Throwable exception() {
- return exception;
+ public Status status() {
+ return suite == Suite.PRODUCTION_TEST && thrown instanceof InconclusiveTestException
+ ? Status.inconclusive
+ : thrown instanceof AssertionError ? Status.failed : Status.error;
}
}
+ public enum Status {
+
+ // Must be kept in order of increasing severity.
+ successful,
+ skipped,
+ aborted,
+ inconclusive,
+ failed,
+ error;
+
+ }
+
/**
* Recursively trims stack traces for the given throwable and its causes/suppressed.
* This is based on the assumption that the relevant stack is anything above the first native
@@ -130,13 +317,25 @@ public class TestReport {
return;
StackTraceElement[] stack = thrown.getStackTrace();
- int i = stack.length;
+ int i = 0;
+ int previousNativeFrame = -1;
+ int cutoff = 0;
boolean rootedInTestFramework = false;
- while (--i > 0 && ! stack[i].isNativeMethod()) // Native method invokes the first user test frame.
+ while (++i < stack.length) {
rootedInTestFramework |= testFrameworkRootClass.equals(stack[i].getClassName());
-
- if (rootedInTestFramework && i > 0)
- thrown.setStackTrace(copyOf(stack, i));
+ if (stack[i].isNativeMethod())
+ previousNativeFrame = i; // Native method invokes the first user test frame.
+ if (rootedInTestFramework && previousNativeFrame > 0) {
+ cutoff = previousNativeFrame;
+ break;
+ }
+ boolean isDynamicTestInvocation = "org.junit.jupiter.engine.descriptor.DynamicTestTestDescriptor".equals(stack[i].getClassName());
+ if (isDynamicTestInvocation) {
+ cutoff = i;
+ break;
+ }
+ }
+ thrown.setStackTrace(copyOf(stack, cutoff));
for (Throwable suppressed : thrown.getSuppressed())
trimStackTraces(suppressed, testFrameworkRootClass);
@@ -144,4 +343,15 @@ public class TestReport {
trimStackTraces(thrown.getCause(), testFrameworkRootClass);
}
+ private static String toString(Suite suite) {
+ if (suite == null) return "Tests";
+ switch (suite) {
+ case SYSTEM_TEST: return "System test";
+ case STAGING_SETUP_TEST: return "Staging setup";
+ case STAGING_TEST: return "Staging test";
+ case PRODUCTION_TEST: return "Production test";
+ default: throw new IllegalArgumentException("unexpected suite '" + suite + "'");
+ }
+ }
+
}
diff --git a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReportGeneratingListener.java b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReportGeneratingListener.java
new file mode 100644
index 00000000000..0d767f5aa8a
--- /dev/null
+++ b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestReportGeneratingListener.java
@@ -0,0 +1,183 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+package com.yahoo.vespa.testrunner;
+
+import com.yahoo.vespa.testrunner.TestReport.Node;
+import com.yahoo.vespa.testrunner.TestReport.Status;
+import com.yahoo.vespa.testrunner.TestRunner.Suite;
+import org.junit.platform.engine.TestExecutionResult;
+import org.junit.platform.engine.reporting.ReportEntry;
+import org.junit.platform.launcher.TestExecutionListener;
+import org.junit.platform.launcher.TestIdentifier;
+import org.junit.platform.launcher.TestPlan;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.time.Clock;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.logging.Handler;
+import java.util.logging.Level;
+import java.util.logging.LogRecord;
+import java.util.logging.Logger;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Objects.requireNonNullElse;
+import static java.util.Objects.requireNonNullElseGet;
+import static java.util.logging.Level.INFO;
+import static java.util.logging.Level.SEVERE;
+import static java.util.logging.Level.WARNING;
+import static java.util.stream.Collectors.joining;
+
+class TestReportGeneratingListener implements TestExecutionListener {
+
+ private final TestReport report; // Holds a structured view of the test run.
+ private final Consumer<LogRecord> logger; // Used to show test output for a plain textual view of the test run.
+ private final TeeStream stdoutTee; // Captures output from test code.
+ private final TeeStream stderrTee; // Captures output from test code.
+ private final Handler handler; // Captures logging from test code.
+ private final Clock clock;
+
+ TestReportGeneratingListener(Suite suite, Consumer<LogRecord> logger, TeeStream stdoutTee, TeeStream stderrTee, Clock clock) {
+ this.report = new TestReport(clock, suite);
+ this.logger = logger;
+ this.stdoutTee = stdoutTee;
+ this.stderrTee = stderrTee;
+ this.handler = new TestReportHandler();
+ this.clock = clock;
+ }
+
+ @Override
+ public void testPlanExecutionStarted(TestPlan testPlan) {
+ report.start(testPlan);
+ stdoutTee.setTee(new LineLoggingOutputStream());
+ stderrTee.setTee(new LineLoggingOutputStream());
+ Logger.getLogger("").addHandler(handler);
+ }
+
+ @Override
+ public void testPlanExecutionFinished(TestPlan testPlan) {
+ Logger.getLogger("").removeHandler(handler);
+ try {
+ stderrTee.clearTee().close();
+ stdoutTee.clearTee().close();
+ }
+ catch (IOException ignored) { } // Doesn't happen.
+
+ TestReport.Node root = report.complete();
+ Level level = INFO;
+ switch (root.status()) {
+ case skipped: case aborted: level = WARNING; break;
+ case failed: case error: level = SEVERE;
+ }
+ Map<Status, Long> tally = root.tally();
+ log(level,
+ "Done running " + tally.values().stream().mapToLong(Long::longValue).sum() + " tests: " +
+ tally.entrySet().stream()
+ .map(entry -> entry.getValue() + " " + entry.getKey())
+ .collect(joining(", ")));
+ }
+
+ @Override
+ public void dynamicTestRegistered(TestIdentifier testIdentifier) {
+ if (testIdentifier.isContainer() && testIdentifier.getParentId().isPresent()) // Skip root engine level.
+ log(INFO, "Registered dynamic container: " + testIdentifier.getDisplayName());
+ if (testIdentifier.isTest())
+ log(INFO, "Registered dynamic test: " + testIdentifier.getDisplayName());
+ }
+
+ @Override
+ public void executionStarted(TestIdentifier testIdentifier) {
+ if (testIdentifier.isContainer() && testIdentifier.getParentId().isPresent()) // Skip root engine level.
+ log(INFO, "Running all tests in: " + testIdentifier.getDisplayName());
+ if (testIdentifier.isTest())
+ log(INFO, "Running test: " + testIdentifier.getDisplayName());
+ report.start(testIdentifier);
+ }
+
+ @Override
+ public void executionSkipped(TestIdentifier testIdentifier, String reason) {
+ log(WARNING, "Skipping: " + testIdentifier.getDisplayName() + ": " + reason);
+ report.skip(testIdentifier);
+ }
+
+ @Override
+ public void executionFinished(TestIdentifier testIdentifier, TestExecutionResult testExecutionResult) {
+ Node node = testExecutionResult.getStatus() == TestExecutionResult.Status.ABORTED
+ ? report.abort(testIdentifier)
+ : report.complete(testIdentifier, testExecutionResult.getThrowable().orElse(null));
+ Status status = node.status();
+ Level level = status.compareTo(Status.failed) >= 0 ? SEVERE : status.compareTo(Status.skipped) >= 0 ? WARNING : INFO;
+
+ if (testIdentifier.isContainer()) {
+ if (testIdentifier.getParentIdObject().isPresent()) {
+ log(level,
+ "Tests in " + testIdentifier.getDisplayName() + " done: " +
+ node.tally().entrySet().stream().map(entry -> entry.getValue() + " " + entry.getKey()).collect(joining(", ")));
+ }
+ }
+ if (testIdentifier.isTest()) {
+ testIdentifier.getParentIdObject().ifPresent(parent -> log(level,
+ "Test " + status + ": " + testIdentifier.getDisplayName(),
+ testExecutionResult.getThrowable().orElse(null)));
+ }
+ }
+
+ @Override
+ public void reportingEntryPublished(TestIdentifier __, ReportEntry report) { // Note: identifier not needed as long as we run serially.
+ Map<String, String> entries = new HashMap<>(report.getKeyValuePairs());
+ Level level = Level.parse(requireNonNullElse(entries.remove("level"), "INFO"));
+ String logger = entries.remove("logger");
+ String message = requireNonNullElseGet(entries.remove("value"), () -> entries.entrySet().stream()
+ .map(entry -> entry.getKey() + ": " + entry.getValue())
+ .collect(joining("\n")));
+
+ LogRecord record = new LogRecord(level, message);
+ record.setInstant(report.getTimestamp().toInstant(ZoneOffset.UTC));
+ record.setLoggerName(logger);
+ handler.publish(record);
+ }
+
+ TestReport report() {
+ return report;
+ }
+
+ private void log(Level level, String message) {
+ log(level, message, null);
+ }
+
+ private void log(Level level, String message, Throwable thrown) {
+ LogRecord record = new LogRecord(level, message);
+ record.setThrown(thrown);
+ logger.accept(record);
+ }
+
+ private class LineLoggingOutputStream extends OutputStream {
+ final ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+ @Override public void write(int b) {
+ if (b == '\n') {
+ handler.publish(new LogRecord(INFO, buffer.toString(UTF_8)));
+ buffer.reset();
+ }
+ else buffer.write(b);
+ }
+ @Override public void close() {
+ if (buffer.size() > 0) write('\n');
+ }
+ }
+
+ private class TestReportHandler extends Handler {
+ @Override public void publish(LogRecord record) {
+ if ("html".equals(record.getLevel().getName())) record.setLevel(INFO);
+ record.setInstant(clock.instant());
+ logger.accept(record);
+ report.log(record);
+ }
+ @Override public void flush() { }
+ @Override public void close() { }
+ }
+
+}
diff --git a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestRunnerHandler.java b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestRunnerHandler.java
index 0fdc88e1ad9..4bf40323193 100644
--- a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestRunnerHandler.java
+++ b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/TestRunnerHandler.java
@@ -12,6 +12,11 @@ import com.yahoo.restapi.MessageResponse;
import com.yahoo.restapi.SlimeJsonResponse;
import com.yahoo.slime.Cursor;
import com.yahoo.slime.Slime;
+import com.yahoo.vespa.testrunner.TestReport.ContainerNode;
+import com.yahoo.vespa.testrunner.TestReport.FailureNode;
+import com.yahoo.vespa.testrunner.TestReport.Node;
+import com.yahoo.vespa.testrunner.TestReport.OutputNode;
+import com.yahoo.vespa.testrunner.TestReport.TestNode;
import com.yahoo.yolean.Exceptions;
import java.io.ByteArrayOutputStream;
@@ -20,12 +25,14 @@ import java.io.PrintStream;
import java.time.ZoneOffset;
import java.time.format.DateTimeFormatter;
import java.util.Collection;
+import java.util.Map;
import java.util.Optional;
import java.util.concurrent.Executor;
import java.util.logging.Level;
import java.util.logging.LogRecord;
import static com.yahoo.jdisc.Response.Status;
+import static java.nio.charset.StandardCharsets.UTF_8;
/**
* @author valerijf
@@ -74,7 +81,6 @@ public class TestRunnerHandler extends ThreadedHttpRequestHandler {
.orElse(-1L);
return new SlimeJsonResponse(logToSlime(testRunner.getLog(fetchRecordsAfter)));
case "/tester/v1/status":
- log.info("Responding with status " + testRunner.getStatus());
return new MessageResponse(testRunner.getStatus().name());
case "/tester/v1/report":
TestReport report = testRunner.getReport();
@@ -139,32 +145,91 @@ public class TestRunnerHandler extends ThreadedHttpRequestHandler {
: "error";
}
- private static Slime toSlime(TestReport testReport) {
+ private static Slime toSlime(TestReport report) {
var slime = new Slime();
var root = slime.setObject();
- if (testReport == null)
- return slime;
+ toSlime(root.setObject("report"), (Node) report.root());
+
+ // TODO jonmv: remove
+ Map<TestReport.Status, Long> tally = report.root().tally();
var summary = root.setObject("summary");
- summary.setLong("success", testReport.successCount);
- summary.setLong("failed", testReport.failedCount);
- summary.setLong("ignored", testReport.ignoredCount);
- summary.setLong("aborted", testReport.abortedCount);
- summary.setLong("inconclusive", testReport.inconclusiveCount);
- var failureRoot = summary.setArray("failures");
- testReport.failures.forEach(failure -> serializeFailure(failure, failureRoot.addObject()));
-
- var output = root.setArray("output");
- for (LogRecord record : testReport.logLines)
- output.addString(formatter.format(record.getInstant().atOffset(ZoneOffset.UTC)) + " " + record.getMessage());
+ summary.setLong("success", tally.getOrDefault(TestReport.Status.successful, 0L));
+ summary.setLong("failed", tally.getOrDefault(TestReport.Status.failed, 0L) + tally.getOrDefault(TestReport.Status.error, 0L));
+ summary.setLong("ignored", tally.getOrDefault(TestReport.Status.skipped, 0L));
+ summary.setLong("aborted", tally.getOrDefault(TestReport.Status.aborted, 0L));
+ summary.setLong("inconclusive", tally.getOrDefault(TestReport.Status.inconclusive, 0L));
+ toSlime(summary.setArray("failures"), root.setArray("output"), report.root());
return slime;
}
- private static void serializeFailure(TestReport.Failure failure, Cursor slime) {
- slime.setString("testName", failure.testId());
- slime.setString("testError",failure.exception().getMessage());
- slime.setString("exception", ExceptionUtils.getStackTraceAsString(failure.exception()));
+ static void toSlime(Cursor failuresArray, Cursor outputArray, Node node) {
+ for (Node child : node.children())
+ TestRunnerHandler.toSlime(failuresArray, outputArray, child);
+
+ if (node instanceof FailureNode) {
+ Cursor failureObject = failuresArray.addObject();
+ failureObject.setString("testName", node.parent.name());
+ failureObject.setString("testError", ((FailureNode) node).thrown().getMessage());
+ failureObject.setString("exception", ExceptionUtils.getStackTraceAsString(((FailureNode) node).thrown()));
+ }
+ if (node instanceof OutputNode)
+ for (LogRecord record : ((OutputNode) node).log())
+ outputArray.addString(formatter.format(record.getInstant().atOffset(ZoneOffset.UTC)) + " " + record.getMessage());
+ }
+
+ static void toSlime(Cursor nodeObject, Node node) {
+ if (node instanceof ContainerNode) toSlime(nodeObject, (ContainerNode) node);
+ if (node instanceof TestNode) toSlime(nodeObject, (TestNode) node);
+ if (node instanceof OutputNode) toSlime(nodeObject, (OutputNode) node);
+ if (node instanceof FailureNode) toSlime(nodeObject, (FailureNode) node);
+
+ if ( ! node.children().isEmpty()) {
+ Cursor childrenArray = nodeObject.setArray("children");
+ for (Node child : node.children)
+ toSlime(childrenArray.addObject(), child);
+ }
+ }
+
+ static void toSlime(Cursor nodeObject, ContainerNode node) {
+ nodeObject.setString("type", "container");
+ nodeObject.setString("name", node.name());
+ nodeObject.setString("status", node.status().name());
+ nodeObject.setLong("start", node.start().toEpochMilli());
+ nodeObject.setLong("end", node.duration().toMillis());
+ }
+
+ static void toSlime(Cursor nodeObject, TestNode node) {
+ nodeObject.setString("type", "test");
+ nodeObject.setString("name", node.name());
+ nodeObject.setString("status", node.status().name());
+ nodeObject.setLong("start", node.start().toEpochMilli());
+ nodeObject.setLong("end", node.duration().toMillis());
+ }
+
+ static void toSlime(Cursor nodeObject, OutputNode node) {
+ nodeObject.setString("type", "output");
+ Cursor childrenArray = nodeObject.setArray("children");
+ for (LogRecord record : node.log()) {
+ Cursor recordObject = childrenArray.addObject();
+ recordObject.setString("message", (record.getLoggerName() == null ? "" : record.getLoggerName() + ": ") + record.getMessage());
+ recordObject.setLong("at", record.getInstant().toEpochMilli());
+ recordObject.setString("level", typeOf(record.getLevel()));
+ if (record.getThrown() != null) recordObject.setString("trace", traceToString(record.getThrown()));
+ }
+ }
+
+ static void toSlime(Cursor nodeObject, FailureNode node) {
+ nodeObject.setString("type", "failure");
+ nodeObject.setString("status", node.status().name());
+ nodeObject.setString("trace", traceToString(node.thrown()));
+ }
+
+ private static String traceToString(Throwable thrown) {
+ ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+ thrown.printStackTrace(new PrintStream(buffer));
+ return buffer.toString(UTF_8);
}
}
diff --git a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/VespaJunitLogListener.java b/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/VespaJunitLogListener.java
deleted file mode 100644
index b49f4e5e431..00000000000
--- a/vespa-osgi-testrunner/src/main/java/com/yahoo/vespa/testrunner/VespaJunitLogListener.java
+++ /dev/null
@@ -1,150 +0,0 @@
-// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-package com.yahoo.vespa.testrunner;
-
-import ai.vespa.hosted.cd.InconclusiveTestException;
-import org.junit.platform.engine.TestExecutionResult;
-import org.junit.platform.engine.UniqueId;
-import org.junit.platform.engine.reporting.ReportEntry;
-import org.junit.platform.launcher.TestExecutionListener;
-import org.junit.platform.launcher.TestIdentifier;
-
-import java.time.ZoneOffset;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.function.Consumer;
-import java.util.logging.Level;
-import java.util.logging.LogRecord;
-
-import static java.util.Collections.emptyNavigableMap;
-import static java.util.Objects.requireNonNull;
-import static java.util.logging.Level.INFO;
-import static java.util.logging.Level.SEVERE;
-import static java.util.logging.Level.WARNING;
-import static java.util.stream.Collectors.joining;
-
-class VespaJunitLogListener implements TestExecutionListener {
-
- private final Map<String, NavigableMap<Status, List<UniqueId>>> results = new ConcurrentSkipListMap<>();
- private final Consumer<LogRecord> logger;
-
- VespaJunitLogListener(Consumer<LogRecord> logger) {
- this.logger = requireNonNull(logger);
- }
-
- @Override
- public void dynamicTestRegistered(TestIdentifier testIdentifier) {
- if (testIdentifier.isContainer() && testIdentifier.getParentId().isPresent()) // Skip root engine level.
- log(INFO, "Registered dynamic container: " + testIdentifier.getDisplayName());
- if (testIdentifier.isTest())
- log(INFO, "Registered dynamic test: " + testIdentifier.getDisplayName());
- }
-
- @Override
- public void executionStarted(TestIdentifier testIdentifier) {
- if (testIdentifier.isContainer() && testIdentifier.getParentId().isPresent()) // Skip root engine level.
- log(INFO, "Running all tests in: " + testIdentifier.getDisplayName());
- if (testIdentifier.isTest())
- log(INFO, "Running test: " + testIdentifier.getDisplayName());
- }
-
- @Override
- public void executionSkipped(TestIdentifier testIdentifier, String reason) {
- log(WARNING, "Skipped: " + testIdentifier.getDisplayName() + ": " + reason);
- if (testIdentifier.isTest())
- testIdentifier.getParentId().ifPresent(parent -> {
- results.computeIfAbsent(parent, __ -> new ConcurrentSkipListMap<>())
- .computeIfAbsent(Status.skipped, __ -> new CopyOnWriteArrayList<>())
- .add(testIdentifier.getUniqueIdObject());
- });
- }
-
- @Override
- public void executionFinished(TestIdentifier testIdentifier, TestExecutionResult testExecutionResult) {
- if (testIdentifier.isContainer()) {
- if (testIdentifier.getParentIdObject().isPresent()) {
- NavigableMap<Status, List<UniqueId>> children = results.getOrDefault(testIdentifier.getUniqueId(), emptyNavigableMap());
- Level level = children.containsKey(Status.failed) ? SEVERE : INFO;
- log(level,
- "Tests in " + testIdentifier.getDisplayName() + " done: " +
- children.entrySet().stream().map(entry -> entry.getValue().size() + " " + entry.getKey()).collect(joining(", ")));
- }
- else {
- Map<Status, List<String>> testResults = new HashMap<>();
- results.forEach((__, results) -> results.forEach((status, tests) -> tests.forEach(test -> testResults.computeIfAbsent(status, ___ -> new ArrayList<>())
- .add(toString(test)))));
- log(INFO, "Done running " + testResults.values().stream().mapToInt(List::size).sum() + " tests.");
- testResults.forEach((status, tests) -> {
- if (status != Status.successful)
- log(status == Status.failed ? SEVERE : status == Status.inconclusive ? INFO : WARNING,
- status.name().substring(0, 1).toUpperCase() + status.name().substring(1) + " tests:\n" + String.join("\n", tests));
- });
- }
- }
- if (testIdentifier.isTest()) {
- Level level;
- Status status;
- if (testExecutionResult.getThrowable().map(InconclusiveTestException.class::isInstance).orElse(false)) {
- level = INFO;
- status = Status.inconclusive;
- }
- else {
- switch (testExecutionResult.getStatus()) {
- case SUCCESSFUL: level = INFO; status = Status.successful; break;
- case ABORTED: level = WARNING; status = Status.aborted; break;
- case FAILED:
- default: level = SEVERE; status = Status.failed; break;
- }
- }
- testIdentifier.getParentId().ifPresent(parent -> {
- results.computeIfAbsent(parent, __ -> new ConcurrentSkipListMap<>())
- .computeIfAbsent(status, __ -> new CopyOnWriteArrayList<>())
- .add(testIdentifier.getUniqueIdObject());
- });
- log(level, "Test " + status + ": " + testIdentifier.getDisplayName(), testExecutionResult.getThrowable().orElse(null));
- }
- }
-
- static String toString(UniqueId testId) {
- return testId.getSegments().stream().skip(1).map(UniqueId.Segment::getValue).collect(joining("."));
- }
-
- @Override
- public void reportingEntryPublished(TestIdentifier testIdentifier, ReportEntry report) {
- String message = report.getKeyValuePairs().keySet().equals(Set.of("value"))
- ? report.getKeyValuePairs().get("value")
- : report.getKeyValuePairs().entrySet().stream()
- .map(entry -> entry.getKey() + ": " + entry.getValue())
- .collect(joining("\n"));
- LogRecord record = new LogRecord(INFO, message);
- record.setInstant(report.getTimestamp().toInstant(ZoneOffset.UTC));
- logger.accept(record);
- }
-
- private void log(Level level, String message) {
- log(level, message, null);
- }
-
- private void log(Level level, String message, Throwable thrown) {
- LogRecord record = new LogRecord(level, message);
- record.setThrown(thrown);
- logger.accept(record);
- }
-
- private enum Status {
-
- successful,
- inconclusive,
- failed,
- aborted,
- skipped;
-
- }
-
-}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/DisabledClassTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/DisabledClassTest.java
new file mode 100644
index 00000000000..795bf8c6a1e
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/DisabledClassTest.java
@@ -0,0 +1,17 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+@Disabled("because")
+@Expect(skipped = 2, status = 1)
+public class DisabledClassTest {
+
+ @Test
+ void test() { }
+
+ @Test
+ void fest() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/DisabledTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/DisabledTest.java
new file mode 100644
index 00000000000..561ec81e865
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/DisabledTest.java
@@ -0,0 +1,14 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+@Expect(skipped = 1, status = 1)
+public class DisabledTest {
+
+ @Test
+ @Disabled("because")
+ void test() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAfterAllTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAfterAllTest.java
new file mode 100644
index 00000000000..4c7132fc01a
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAfterAllTest.java
@@ -0,0 +1,19 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Test;
+
+@Expect(successful = 2, status = 5)
+public class FailingAfterAllTest {
+
+ @AfterAll
+ static void fail() { throw new RuntimeException(); }
+
+ @Test
+ void test() { }
+
+ @Test
+ void fest() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAfterEachTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAfterEachTest.java
new file mode 100644
index 00000000000..b1ec3cb13fd
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAfterEachTest.java
@@ -0,0 +1,16 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+
+@Expect(error = 1, status = 5)
+public class FailingAfterEachTest {
+
+ @AfterEach
+ void fail() { throw new RuntimeException(); }
+
+ @Test
+ void test() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAssertionTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAssertionTest.java
new file mode 100644
index 00000000000..4dd8be898ec
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAssertionTest.java
@@ -0,0 +1,13 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+@Expect(failed = 1, status = 4)
+public class FailingAssertionTest {
+
+ @Test
+ void fail() { Assertions.fail(); }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAssumptionTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAssumptionTest.java
new file mode 100644
index 00000000000..1b542a7dd7d
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingAssumptionTest.java
@@ -0,0 +1,14 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
+
+@Expect(aborted = 1, status = 2)
+public class FailingAssumptionTest {
+
+ @Test
+ void test() { assumeTrue(false, "assumption"); }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllAssertionTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllAssertionTest.java
new file mode 100644
index 00000000000..15e67f2c51c
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllAssertionTest.java
@@ -0,0 +1,20 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+@Expect(skipped = 2, status = 4)
+public class FailingBeforeAllAssertionTest {
+
+ @BeforeAll
+ static void fail() { Assertions.fail(); }
+
+ @Test
+ void test() { }
+
+ @Test
+ void fest() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllTest.java
new file mode 100644
index 00000000000..ae26b3fd038
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllTest.java
@@ -0,0 +1,19 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+@Expect(skipped = 2, status = 5)
+public class FailingBeforeAllTest {
+
+ @BeforeAll
+ static void fail() { throw new RuntimeException(); }
+
+ @Test
+ void test() { }
+
+ @Test
+ void fest() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllTestFactoryTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllTestFactoryTest.java
new file mode 100644
index 00000000000..89457b145c9
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeAllTestFactoryTest.java
@@ -0,0 +1,26 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+@Expect(skipped = 1, status = 5)
+public class FailingBeforeAllTestFactoryTest {
+
+ @BeforeAll
+ static void fail() { throw new RuntimeException(); }
+
+ @TestFactory
+ Stream<DynamicTest> tests() {
+ return IntStream.range(0, 3).mapToObj(i -> DynamicTest.dynamicTest("test-" + i, () -> { }));
+ }
+
+ @Test
+ void fest() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeEachTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeEachTest.java
new file mode 100644
index 00000000000..5e5ebe47c99
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingBeforeEachTest.java
@@ -0,0 +1,16 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+@Expect(error = 1, status = 5)
+public class FailingBeforeEachTest {
+
+ @BeforeEach
+ void fail() { throw new RuntimeException(); }
+
+ @Test
+ void test() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingClassAssumptionTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingClassAssumptionTest.java
new file mode 100644
index 00000000000..2a1085c3db3
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingClassAssumptionTest.java
@@ -0,0 +1,19 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Assumptions;
+import org.junit.jupiter.api.Test;
+
+@Expect(aborted = 2, status = 2)
+public class FailingClassAssumptionTest {
+
+ { Assumptions.assumeTrue(false, "assumption"); }
+
+ @Test
+ void test() { }
+
+ @Test
+ void fest() { }
+
+}
+
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingClassLoadingTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingClassLoadingTest.java
new file mode 100644
index 00000000000..fc2b33ee03f
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingClassLoadingTest.java
@@ -0,0 +1,19 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+@Expect(failed = 1, error = 1, status = 5)
+public class FailingClassLoadingTest {
+
+ static { Assertions.fail(); }
+
+ @Test
+ void test() { }
+
+ @Test
+ void fest() { }
+
+}
+
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingExtensionTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingExtensionTest.java
new file mode 100644
index 00000000000..68e348a730a
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingExtensionTest.java
@@ -0,0 +1,23 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.Extension;
+
+import static java.util.Objects.requireNonNull;
+
+@Expect(error = 1, status = 5)
+public class FailingExtensionTest {
+
+ @Test
+ @ExtendWith(FailingExtension.class)
+ void test() { }
+
+ static class FailingExtension implements Extension {
+
+ { if (true) throw new NullPointerException(); }
+
+ }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInnerClassTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInnerClassTest.java
new file mode 100644
index 00000000000..85ed49fcf0b
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInnerClassTest.java
@@ -0,0 +1,28 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.fail;
+
+@Expect(successful = 1, failed = 1, status = 4)
+public class FailingInnerClassTest {
+
+ @Nested
+ class Failing {
+
+ @Test
+ void test() { fail(); }
+
+ }
+
+ @Nested
+ class Succeeding {
+
+ @Test
+ void test() { }
+
+ }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationAssertionTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationAssertionTest.java
new file mode 100644
index 00000000000..87c19872f13
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationAssertionTest.java
@@ -0,0 +1,19 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.fail;
+
+@Expect(failed = 2, status = 4)
+public class FailingInstantiationAssertionTest {
+
+ { fail(); }
+
+ @Test
+ void test() { }
+
+ @Test
+ void fest() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationTest.java
index 4c4e5bf3e36..50e0c6a43b7 100644
--- a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationTest.java
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingInstantiationTest.java
@@ -1,9 +1,9 @@
package com.yahoo.vespa.test.samples;
-import ai.vespa.hosted.cd.SystemTest;
+import com.yahoo.vespa.testrunner.Expect;
import org.junit.jupiter.api.Test;
-@SystemTest
+@Expect(error = 2, status = 5)
public class FailingInstantiationTest {
final int i = Integer.parseInt("");
@@ -11,4 +11,7 @@ public class FailingInstantiationTest {
@Test
void test() { }
+ @Test
+ void fest() { }
+
}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTest.java
new file mode 100644
index 00000000000..f2a65c58728
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTest.java
@@ -0,0 +1,12 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+
+@Expect(error = 1, status = 5)
+public class FailingTest {
+
+ @Test
+ void test() { throw new RuntimeException(); }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTestAndBothAftersTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTestAndBothAftersTest.java
new file mode 100644
index 00000000000..5ca1f43b976
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTestAndBothAftersTest.java
@@ -0,0 +1,21 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+@Expect(failed = 1, status = 5)
+public class FailingTestAndBothAftersTest {
+
+ @AfterAll
+ static void fail() { throw new RuntimeException(); }
+
+ @AfterEach
+ void moreFail() { throw new RuntimeException(); }
+
+ @Test
+ void test() { Assertions.fail(); }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTestFactoryTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTestFactoryTest.java
new file mode 100644
index 00000000000..fa7a39eea7d
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/FailingTestFactoryTest.java
@@ -0,0 +1,26 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+@Expect(skipped = 1, status = 5)
+public class FailingTestFactoryTest {
+
+ @BeforeAll
+ static void fail() { throw new RuntimeException(); }
+
+ @TestFactory
+ Stream<DynamicTest> tests() {
+ return IntStream.range(0, 3).mapToObj(i -> DynamicTest.dynamicTest("test-" + i, () -> { throw new RuntimeException("error"); }));
+ }
+
+ @Test
+ void fest() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/InconclusiveTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/InconclusiveTest.java
new file mode 100644
index 00000000000..868568e8bb5
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/InconclusiveTest.java
@@ -0,0 +1,15 @@
+package com.yahoo.vespa.test.samples;
+
+import ai.vespa.hosted.cd.InconclusiveTestException;
+import ai.vespa.hosted.cd.ProductionTest;
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+
+@ProductionTest
+@Expect(inconclusive = 1, status = 3)
+public class InconclusiveTest {
+
+ @Test
+ void test() { throw new InconclusiveTestException("soon"); }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/NotInconclusiveTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/NotInconclusiveTest.java
new file mode 100644
index 00000000000..fea1e827260
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/NotInconclusiveTest.java
@@ -0,0 +1,13 @@
+package com.yahoo.vespa.test.samples;
+
+import ai.vespa.hosted.cd.InconclusiveTestException;
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+
+@Expect(error = 1, status = 5)
+public class NotInconclusiveTest {
+
+ @Test
+ void test() { throw new InconclusiveTestException("soon"); }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/SampleTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/SampleTest.java
new file mode 100644
index 00000000000..bc878353d4b
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/SampleTest.java
@@ -0,0 +1,119 @@
+package com.yahoo.vespa.test.samples;
+
+import ai.vespa.hosted.cd.InconclusiveTestException;
+import ai.vespa.hosted.cd.ProductionTest;
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Assumptions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.ClassOrderer;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestClassOrder;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.TestMethodOrder;
+import org.junit.jupiter.api.TestReporter;
+
+import java.util.logging.ConsoleHandler;
+import java.util.logging.Handler;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+import java.util.stream.Stream;
+
+import static java.util.logging.Level.FINE;
+import static java.util.logging.Level.INFO;
+import static java.util.logging.Level.WARNING;
+import static org.junit.jupiter.api.DynamicTest.dynamicTest;
+
+@ProductionTest
+@Expect(successful = 3, skipped = 2, aborted = 1, inconclusive = 1, failed = 2, error = 1, status = 5)
+@TestMethodOrder(MethodOrderer.DisplayName.class)
+@TestClassOrder(ClassOrderer.DisplayName.class)
+public class SampleTest {
+
+ static Handler consoleHandler = null;
+
+ @BeforeAll
+ static void setupLogging() {
+ Handler[] handlers = Logger.getLogger("").getHandlers();
+ for (Handler handler : handlers)
+ if (handler instanceof ConsoleHandler)
+ consoleHandler = handler;
+ Logger.getLogger("").removeHandler(consoleHandler);
+ }
+
+ @AfterAll
+ static void restoreLogging() {
+ Logger.getLogger("").addHandler(consoleHandler);
+ }
+
+ private static final Logger log = Logger.getLogger(SampleTest.class.getName());
+
+ @BeforeEach
+ void spam() {
+ System.err.println("spam");
+ }
+
+ @Test
+ @Disabled("disabled for test purposes")
+ void ignored() { }
+
+ @Test
+ void aborted() {
+ Assumptions.assumeTrue(false, "thou shalt not pass!");
+ }
+
+ @Test
+ void successful() {
+ log.log(new Level("html", INFO.intValue()) { }, "<body />");
+ log.log(INFO, "Very informative");
+ log.log(WARNING, "Oh no", new IllegalArgumentException("error", new RuntimeException("wrapped")));
+ }
+
+ @Test
+ void failing() {
+ log.log(INFO, "I have a bad feeling about this");
+ Assertions.assertEquals("foo", "bar", "baz");
+ }
+
+ @Test
+ void error() {
+ log.log(FINE, "What could possibly go wrong this time?");
+ throw new NoClassDefFoundError();
+ }
+
+ @Test
+ void inconclusive(TestReporter reporter) {
+ reporter.publishEntry("I'm here with Erwin today; Erwin, what can you tell us about your cat?");
+ throw new InconclusiveTestException("the cat is both dead _and_ alive");
+ }
+
+ @Nested
+ class Inner {
+
+ @Test
+ void first() { }
+
+ @TestFactory
+ Stream<DynamicTest> others() {
+ return Stream.of(dynamicTest("second", () -> System.out.println("Catch me if you can!")),
+ dynamicTest("third", () -> Assertions.fail("no charm")));
+ }
+
+ }
+
+ @Nested
+ @Disabled
+ class Skipped {
+
+ @Test
+ void disabled() { }
+
+ }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/SucceedingTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/SucceedingTest.java
new file mode 100644
index 00000000000..59a56a1c9c7
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/SucceedingTest.java
@@ -0,0 +1,13 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+
+@Expect(successful = 1, status = 0)
+public class SucceedingTest {
+
+ @Test
+ void test() { }
+
+}
+
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/TimingOutTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/TimingOutTest.java
new file mode 100644
index 00000000000..b248fe065fb
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/TimingOutTest.java
@@ -0,0 +1,18 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.concurrent.TimeUnit;
+
+@Expect(error = 1, status = 5)
+public class TimingOutTest {
+
+ @Test
+ @Timeout(value = 1, unit = TimeUnit.MILLISECONDS)
+ void test() throws InterruptedException {
+ Thread.sleep(10_000);
+ }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/UsingTestRuntimeTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/UsingTestRuntimeTest.java
new file mode 100644
index 00000000000..acb41bb42be
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/UsingTestRuntimeTest.java
@@ -0,0 +1,22 @@
+package com.yahoo.vespa.test.samples;
+
+import ai.vespa.cloud.Environment;
+import ai.vespa.hosted.cd.TestRuntime;
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+@Expect(successful = 1, status = 0)
+public class UsingTestRuntimeTest {
+
+ @Test
+ void testTestRuntime() {
+ TestRuntime runtime = TestRuntime.get();
+ assertEquals(Environment.test, runtime.zone().environment());
+ assertEquals("name", runtime.zone().region());
+ assertNull(runtime.deploymentToTest().endpoint("dummy"));
+ }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/WrongBeforeAllTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/WrongBeforeAllTest.java
new file mode 100644
index 00000000000..842ce89e63a
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/test/samples/WrongBeforeAllTest.java
@@ -0,0 +1,19 @@
+package com.yahoo.vespa.test.samples;
+
+import com.yahoo.vespa.testrunner.Expect;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+@Expect(skipped = 2, status = 5)
+public class WrongBeforeAllTest {
+
+ @BeforeAll
+ void wrong() { }
+
+ @Test
+ void test() { }
+
+ @Test
+ void fest() { }
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/AggregateTestRunnerTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/AggregateTestRunnerTest.java
index 8bd72d35737..52a96521dc1 100644
--- a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/AggregateTestRunnerTest.java
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/AggregateTestRunnerTest.java
@@ -2,9 +2,13 @@
package com.yahoo.vespa.testrunner;
import com.yahoo.exception.ExceptionUtils;
-import com.yahoo.vespa.testrunner.TestReport.Failure;
+import com.yahoo.vespa.test.samples.SampleTest;
+import com.yahoo.vespa.testrunner.TestReport.Node;
+import com.yahoo.vespa.testrunner.TestReport.Status;
+import com.yahoo.vespa.testrunner.TestRunner.Suite;
import org.junit.jupiter.api.Test;
+import java.time.Clock;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@@ -32,6 +36,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
*/
class AggregateTestRunnerTest {
+ static final TestReport report = JunitRunnerTest.test(Suite.PRODUCTION_TEST, new byte[0], SampleTest.class).getReport();
+
@Test
void onlySupportedRunnersAreUsed() {
MockTestRunner unsupported = new MockTestRunner();
@@ -125,15 +131,6 @@ class AggregateTestRunnerTest {
// Verify reports are merged.
assertNull(runner.getReport());
- Failure failure = new Failure("test", null);
- TestReport report = TestReport.builder()
- .withLogs(List.of(record1))
- .withFailures(List.of(failure))
- .withSuccessCount(8)
- .withIgnoredCount(4)
- .withFailedCount(2)
- .withAbortedCount(1)
- .build();
first.report = report;
assertSame(report, runner.getReport());
second.report = report;
@@ -141,24 +138,19 @@ class AggregateTestRunnerTest {
second.future.complete(null);
TestReport merged = runner.getReport();
- assertEquals(List.of(record1, record1), merged.logLines);
- assertEquals(List.of(failure, failure), merged.failures);
- assertEquals(16, merged.successCount);
- assertEquals(8, merged.ignoredCount);
- assertEquals(4, merged.failedCount);
- assertEquals(2, merged.abortedCount);
-
+ List<Node> expected = new ArrayList<>(first.report.root().children());
+ expected.addAll(second.report.root().children());
+ assertEquals(expected, new ArrayList<>(merged.root().children()));
+
+ for (Status status : Status.values())
+ assertEquals( first.report.root().tally().getOrDefault(status, 0L)
+ + second.report.root().tally().getOrDefault(status, 0L),
+ merged.root().tally().getOrDefault(status, 0L));
}
@Test
void testReportStatus() {
- assertEquals(NO_TESTS, TestReport.builder().build().status());
- assertEquals(SUCCESS, TestReport.builder().withSuccessCount(1).build().status());
- assertEquals(INCONCLUSIVE, TestReport.builder().withSuccessCount(1).withInconclusiveCount(1).build().status());
- assertEquals(FAILURE, TestReport.builder().withSuccessCount(1).withFailedCount(1).build().status());
- assertEquals(NO_TESTS, TestReport.builder().withAbortedCount(1).build().status());
- assertEquals(NO_TESTS, TestReport.builder().withIgnoredCount(1).build().status());
- assertEquals(FAILURE, JunitRunner.createReportWithFailedInitialization(new RuntimeException("hello")).status());
+ assertEquals(FAILURE, JunitRunner.testRunnerStatus(TestReport.createFailed(Clock.systemUTC(), Suite.SYSTEM_TEST, new RuntimeException("hello"))));
}
@Test
@@ -172,11 +164,11 @@ class AggregateTestRunnerTest {
}
}
catch (Exception e) {
- TestReport.trimStackTraces(e, "org.junit.platform.launcher.core.SessionPerRequestLauncher");
+ TestReport.trimStackTraces(e, "org.junit.platform.commons.util.ReflectionUtils");
assertEquals("java.lang.RuntimeException: java.lang.RuntimeException: inner\n" +
- "\tat com.yahoo.vespa.testrunner.AggregateTestRunnerTest.testStackTrimming(AggregateTestRunnerTest.java:171)\n" +
+ "\tat com.yahoo.vespa.testrunner.AggregateTestRunnerTest.testStackTrimming(AggregateTestRunnerTest.java:163)\n" +
"Caused by: java.lang.RuntimeException: inner\n" +
- "\tat com.yahoo.vespa.testrunner.AggregateTestRunnerTest.testStackTrimming(AggregateTestRunnerTest.java:168)\n",
+ "\tat com.yahoo.vespa.testrunner.AggregateTestRunnerTest.testStackTrimming(AggregateTestRunnerTest.java:160)\n",
ExceptionUtils.getStackTraceAsString(e));
}
}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/Expect.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/Expect.java
new file mode 100644
index 00000000000..88278b3feb6
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/Expect.java
@@ -0,0 +1,29 @@
+package com.yahoo.vespa.testrunner;
+
+import org.junit.jupiter.api.Tag;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * @author jonmv
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Tag("integration")
+public @interface Expect {
+
+ int status();
+
+ long aborted() default 0;
+
+ long skipped() default 0;
+
+ long successful() default 0;
+
+ long inconclusive() default 0;
+
+ long failed() default 0;
+
+ long error() default 0;
+
+}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/JunitRunnerTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/JunitRunnerTest.java
index 64a4c3dbc80..8e330b0228e 100644
--- a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/JunitRunnerTest.java
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/JunitRunnerTest.java
@@ -1,8 +1,13 @@
package com.yahoo.vespa.testrunner;
-import com.yahoo.vespa.test.samples.FailingInstantiationTest;
-import com.yahoo.vespa.testrunner.TestRunner.Status;
-import org.junit.jupiter.api.Test;
+import ai.vespa.cloud.Environment;
+import ai.vespa.cloud.Zone;
+import ai.vespa.hosted.cd.Deployment;
+import ai.vespa.hosted.cd.TestRuntime;
+import com.yahoo.vespa.testrunner.TestReport.Status;
+import com.yahoo.vespa.testrunner.TestRunner.Suite;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.TestFactory;
import org.junit.jupiter.engine.JupiterTestEngine;
import org.junit.platform.engine.EngineExecutionListener;
import org.junit.platform.engine.TestDescriptor;
@@ -17,12 +22,25 @@ import org.junit.platform.launcher.core.EngineDiscoveryOrchestrator;
import org.junit.platform.launcher.core.EngineExecutionOrchestrator;
import org.junit.platform.launcher.core.LauncherDiscoveryResult;
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.lang.annotation.Annotation;
+import java.time.Clock;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.util.EnumMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Stream;
+import static ai.vespa.hosted.cd.internal.TestRuntimeProvider.testRuntime;
+import static java.util.Objects.requireNonNull;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.jupiter.api.DynamicTest.dynamicTest;
import static org.junit.platform.launcher.core.EngineDiscoveryOrchestrator.Phase.EXECUTION;
/**
@@ -30,24 +48,70 @@ import static org.junit.platform.launcher.core.EngineDiscoveryOrchestrator.Phase
*/
class JunitRunnerTest {
- @Test
- void test() throws ExecutionException, InterruptedException {
- AtomicReference<byte[]> testRuntime = new AtomicReference<>();
- JunitRunner runner = new JunitRunner(testRuntime::set,
- __ -> List.of(FailingInstantiationTest.class),
- this::execute);
-
- runner.test(null, null).get();
- assertEquals(Status.FAILURE, runner.getStatus());
- assertEquals(0, runner.getReport().successCount);
- assertEquals("java.lang.NumberFormatException: For input string: \"\"",
- runner.getReport().failures.get(0).exception().toString());
+ static final Clock clock = Clock.fixed(Instant.EPOCH, ZoneId.of("UTC"));
+
+ @TestFactory
+ Stream<DynamicTest> runSampleTests() {
+ String packageName = "com.yahoo.vespa.test.samples";
+ InputStream classes = getClass().getClassLoader().getResourceAsStream(packageName.replace(".", "/"));
+ BufferedReader reader = new BufferedReader(new InputStreamReader(requireNonNull(classes, packageName + " should contain sample tests")));
+ return reader.lines()
+ .filter(line -> line.endsWith("Test.class"))
+ .map(name -> {
+ try {
+ Class<?> testClass = getClass().getClassLoader().loadClass(packageName + "." + name.replace(".class", ""));
+ return dynamicTest(testClass.getSimpleName(), () -> verify(testClass));
+ }
+ catch (ClassNotFoundException e) {
+ throw new IllegalStateException(e);
+ }
+ });
+ }
+
+ static void verify(Class<?> testClass) {
+ Expect expected = requireNonNull(testClass.getAnnotation(Expect.class), "sample tests must be annotated with @Expect");
+ TestReport report = test(getSuite(testClass), new byte[0], testClass).getReport();
+ assertEquals(Status.values()[expected.status()], report.root().status());
+ Map<Status, Long> tally = new EnumMap<>(Status.class);
+ if (expected.successful() > 0) tally.put(Status.successful, expected.successful());
+ if (expected.skipped() > 0) tally.put(Status.skipped, expected.skipped());
+ if (expected.aborted() > 0) tally.put(Status.aborted, expected.aborted());
+ if (expected.inconclusive() > 0) tally.put(Status.inconclusive, expected.inconclusive());
+ if (expected.failed() > 0) tally.put(Status.failed, expected.failed());
+ if (expected.error() > 0) tally.put(Status.error, expected.error());
+ assertEquals(tally, report.root().tally());
+ }
+
+ static Suite getSuite(Class<?> testClass) {
+ for (Annotation annotation : testClass.getAnnotations()) {
+ switch (annotation.annotationType().getSimpleName()) {
+ case "SystemTest": return Suite.SYSTEM_TEST;
+ case "StagingSetup": return Suite.STAGING_SETUP_TEST;
+ case "StagingTest": return Suite.STAGING_TEST;
+ case "ProductionTest": return Suite.PRODUCTION_TEST;
+ }
+ }
+ return null;
+ }
+
+ static TestRunner test(Suite suite, byte[] testConfig, Class<?>... testClasses) {
+ JunitRunner runner = new JunitRunner(clock,
+ config -> { assertSame(testConfig, config); testRuntime.set(new MockTestRuntime()); },
+ __ -> List.of(testClasses),
+ JunitRunnerTest::execute);
+ try {
+ runner.test(suite, testConfig).get();
+ }
+ catch (Exception e) {
+ fail(e);
+ }
+ return runner;
}
// For some inane reason, the JUnit test framework makes it impossible to simply launch a new instance of itself
- // from inside a unit test (run by itself) in the standard way, so all this kludge is necessary to work around that.
- void execute(LauncherDiscoveryRequest discoveryRequest, TestExecutionListener... listeners) {
+ // from inside a unit test (run by itself) in the standard way, so this kludge is necessary to work around that.
+ static void execute(LauncherDiscoveryRequest discoveryRequest, TestExecutionListener... listeners) {
TestEngine testEngine = new JupiterTestEngine();
LauncherDiscoveryResult discoveryResult = new EngineDiscoveryOrchestrator(Set.of(testEngine), Set.of()).discover(discoveryRequest, EXECUTION);
TestDescriptor engineTestDescriptor = discoveryResult.getEngineTestDescriptor(testEngine);
@@ -100,4 +164,19 @@ class JunitRunnerTest {
}
+
+ static class MockTestRuntime implements TestRuntime {
+
+ @Override
+ public Deployment deploymentToTest() {
+ return __ -> null;
+ }
+
+ @Override
+ public Zone zone() {
+ return new Zone(Environment.test, "name");
+ }
+
+ }
+
}
diff --git a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/TestRunnerHandlerTest.java b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/TestRunnerHandlerTest.java
index 49fadebe58b..5ce737d7649 100644
--- a/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/TestRunnerHandlerTest.java
+++ b/vespa-osgi-testrunner/src/test/java/com/yahoo/vespa/testrunner/TestRunnerHandlerTest.java
@@ -5,6 +5,12 @@ import com.yahoo.component.ComponentId;
import com.yahoo.component.provider.ComponentRegistry;
import com.yahoo.container.jdisc.HttpRequest;
import com.yahoo.container.jdisc.HttpResponse;
+import com.yahoo.slime.Inspector;
+import com.yahoo.vespa.test.samples.FailingExtensionTest;
+import com.yahoo.vespa.test.samples.FailingTestAndBothAftersTest;
+import com.yahoo.vespa.test.samples.WrongBeforeAllTest;
+import com.yahoo.vespa.testrunner.TestReport.Node;
+import com.yahoo.vespa.testrunner.TestReport.OutputNode;
import com.yahoo.vespa.testrunner.TestRunner.Status;
import com.yahoo.vespa.testrunner.TestRunner.Suite;
import org.junit.jupiter.api.BeforeEach;
@@ -12,14 +18,17 @@ import org.junit.jupiter.api.Test;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.time.Clock;
import java.time.Instant;
+import java.time.ZoneId;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.logging.Level;
import java.util.logging.LogRecord;
-import java.util.stream.Collectors;
import static com.yahoo.jdisc.http.HttpRequest.Method.GET;
import static com.yahoo.slime.SlimeUtils.jsonToSlimeOrThrow;
@@ -29,6 +38,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
/**
* @author mortent
+ * @author jonmv
*/
class TestRunnerHandlerTest {
@@ -39,19 +49,18 @@ class TestRunnerHandlerTest {
@BeforeEach
void setup() {
- List<LogRecord> logRecords = List.of(logRecord("Tests started"));
- Throwable exception = new RuntimeException("org.junit.ComparisonFailure: expected:<foo> but was:<bar>");
- exception.setStackTrace(new StackTraceElement[]{new StackTraceElement("Foo", "bar", "Foo.java", 1123)});
- TestReport testReport = TestReport.builder()
- .withSuccessCount(1)
- .withFailedCount(2)
- .withIgnoredCount(3)
- .withAbortedCount(4)
- .withInconclusiveCount(5)
- .withFailures(List.of(new TestReport.Failure("Foo.bar()", exception)))
- .withLogs(logRecords).build();
-
- aggregateRunner = AggregateTestRunner.of(List.of(new MockRunner(TestRunner.Status.SUCCESS, testReport)));
+ TestReport moreTestsReport = JunitRunnerTest.test(Suite.PRODUCTION_TEST,
+ new byte[0],
+ FailingTestAndBothAftersTest.class,
+ WrongBeforeAllTest.class,
+ FailingExtensionTest.class)
+ .getReport();
+ TestReport failedReport = TestReport.createFailed(Clock.fixed(testInstant, ZoneId.of("UTC")),
+ Suite.PRODUCTION_TEST,
+ new ClassNotFoundException("School's out all summer!"));
+ aggregateRunner = AggregateTestRunner.of(List.of(new MockRunner(TestRunner.Status.SUCCESS,
+ AggregateTestRunnerTest.report.mergedWith(moreTestsReport)
+ .mergedWith(failedReport))));
testRunnerHandler = new TestRunnerHandler(Executors.newSingleThreadExecutor(), aggregateRunner);
}
@@ -61,7 +70,7 @@ class TestRunnerHandlerTest {
HttpResponse response = testRunnerHandler.handle(HttpRequest.createTestRequest("http://localhost:1234/tester/v1/report", GET));
ByteArrayOutputStream out = new ByteArrayOutputStream();
response.render(out);
- assertEquals(new String(toJsonBytes(jsonToSlimeOrThrow("{\"summary\":{\"success\":1,\"failed\":2,\"ignored\":3,\"aborted\":4,\"inconclusive\":5,\"failures\":[{\"testName\":\"Foo.bar()\",\"testError\":\"org.junit.ComparisonFailure: expected:<foo> but was:<bar>\",\"exception\":\"java.lang.RuntimeException: org.junit.ComparisonFailure: expected:<foo> but was:<bar>\\n\\tat Foo.bar(Foo.java:1123)\\n\"}]},\"output\":[\"00:00:12.000 Tests started\"]}").get(), false), UTF_8),
+ assertEquals(new String(toJsonBytes(jsonToSlimeOrThrow(readTestResource("/report.json")).get(), false), UTF_8),
new String(toJsonBytes(jsonToSlimeOrThrow(out.toByteArray()).get(), false), UTF_8));
}
@@ -73,16 +82,39 @@ class TestRunnerHandlerTest {
HttpResponse response = testRunnerHandler.handle(HttpRequest.createTestRequest("http://localhost:1234/tester/v1/log", GET));
ByteArrayOutputStream out = new ByteArrayOutputStream();
response.render(out);
- assertEquals(new String(toJsonBytes(jsonToSlimeOrThrow("{\"logRecords\":[{\"id\":0,\"at\":12000,\"type\":\"info\",\"message\":\"Tests started\"}]}").get(), false), UTF_8),
- new String(toJsonBytes(jsonToSlimeOrThrow(out.toByteArray()).get(), false), UTF_8));
+ Inspector actualRoot = jsonToSlimeOrThrow(out.toByteArray()).get();
+ Inspector expectedRoot = jsonToSlimeOrThrow(readTestResource("/output.json")).get();
+ boolean ok = expectedRoot.field("logRecords").entries() == actualRoot.field("logRecords").entries();
+ long last = Long.MIN_VALUE;
+ // Need custom comparison, because sequence ID may be influenced by other tests.
+ for (int i = 0; i < expectedRoot.field("logRecords").entries(); i++) {
+ Inspector expectedEntry = expectedRoot.field("logRecords").entry(i);
+ Inspector actualEntry = actualRoot.field("logRecords").entry(i);
+ ok &= expectedEntry.field("at").equalTo(actualEntry.field("at"));
+ ok &= expectedEntry.field("type").equalTo(actualEntry.field("type"));
+ ok &= expectedEntry.field("message").equalTo(actualEntry.field("message"));
+ last = Math.max(last, actualEntry.field("id").asLong());
+ }
+ if ( ! ok)
+ assertEquals(new String(toJsonBytes(expectedRoot, false), UTF_8),
+ new String(toJsonBytes(actualRoot, false), UTF_8));
// Should not get old log
- response = testRunnerHandler.handle(HttpRequest.createTestRequest("http://localhost:1234/tester/v1/log?after=0", GET));
+ response = testRunnerHandler.handle(HttpRequest.createTestRequest("http://localhost:1234/tester/v1/log?after=" + last, GET));
out = new ByteArrayOutputStream();
response.render(out);
assertEquals("{\"logRecords\":[]}", out.toString(UTF_8));
}
+ static byte[] readTestResource(String name) {
+ try {
+ return TestRunnerHandlerTest.class.getResourceAsStream(name).readAllBytes();
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
@Test
public void returnsEmptyResponsesWhenReportNotReady() throws IOException {
testRunnerHandler = new TestRunnerHandler(Executors.newSingleThreadExecutor(),
@@ -130,10 +162,19 @@ class TestRunnerHandlerTest {
@Override
public Collection<LogRecord> getLog(long after) {
- return getReport() == null ? List.of()
- : getReport().logLines().stream()
- .filter(entry -> entry.getSequenceNumber() > after)
- .collect(Collectors.toList());
+ List<LogRecord> log = new ArrayList<>();
+ if (testReport != null) addLog(log, testReport.root(), after);
+ return log;
+ }
+
+ private void addLog(List<LogRecord> log, Node node, long after) {
+ if (node instanceof OutputNode)
+ for (LogRecord record : ((OutputNode) node).log())
+ if (record.getSequenceNumber() > after)
+ log.add(record);
+
+ for (Node child : node.children())
+ addLog(log, child, after);
}
@Override
diff --git a/vespa-osgi-testrunner/src/test/resources/output.json b/vespa-osgi-testrunner/src/test/resources/output.json
new file mode 100644
index 00000000000..04682fa9f31
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/resources/output.json
@@ -0,0 +1,82 @@
+{
+ "logRecords": [
+ {
+ "id": 3,
+ "at": 0,
+ "type": "info",
+ "message": "spam"
+ },
+ {
+ "id": 6,
+ "at": 0,
+ "type": "info",
+ "message": "spam"
+ },
+ {
+ "id": 9,
+ "at": 0,
+ "type": "info",
+ "message": "spam"
+ },
+ {
+ "id": 10,
+ "at": 0,
+ "type": "info",
+ "message": "I have a bad feeling about this"
+ },
+ {
+ "id": 14,
+ "at": 0,
+ "type": "info",
+ "message": "spam"
+ },
+ {
+ "id": 15,
+ "at": 0,
+ "type": "info",
+ "message": "I'm here with Erwin today; Erwin, what can you tell us about your cat?"
+ },
+ {
+ "id": 18,
+ "at": 0,
+ "type": "info",
+ "message": "spam"
+ },
+ {
+ "id": 19,
+ "at": 0,
+ "type": "info",
+ "message": "<body />"
+ },
+ {
+ "id": 20,
+ "at": 0,
+ "type": "info",
+ "message": "Very informative"
+ },
+ {
+ "id": 21,
+ "at": 0,
+ "type": "warning",
+ "message": "Oh no\njava.lang.IllegalArgumentException: error\n\tat com.yahoo.vespa.test.samples.SampleTest.successful(SampleTest.java:75)\nCaused by: java.lang.RuntimeException: wrapped\n\t... 1 more\n"
+ },
+ {
+ "id": 25,
+ "at": 0,
+ "type": "info",
+ "message": "spam"
+ },
+ {
+ "id": 28,
+ "at": 0,
+ "type": "info",
+ "message": "Catch me if you can!"
+ },
+ {
+ "id": 35,
+ "at": 0,
+ "type": "info",
+ "message": "spam"
+ }
+ ]
+}
diff --git a/vespa-osgi-testrunner/src/test/resources/report.json b/vespa-osgi-testrunner/src/test/resources/report.json
new file mode 100644
index 00000000000..69b11b40ed9
--- /dev/null
+++ b/vespa-osgi-testrunner/src/test/resources/report.json
@@ -0,0 +1,430 @@
+{
+ "report": {
+ "type": "container",
+ "name": "Production test",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "container",
+ "name": "JUnit Jupiter",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "container",
+ "name": "SampleTest",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "test",
+ "name": "aborted()",
+ "status": "aborted",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "output",
+ "children": [
+ {
+ "message": "spam",
+ "at": 0,
+ "level": "info"
+ }
+ ]
+ }
+ ]
+ },
+ {
+ "type": "test",
+ "name": "error()",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "output",
+ "children": [
+ {
+ "message": "spam",
+ "at": 0,
+ "level": "info"
+ }
+ ]
+ },
+ {
+ "type": "failure",
+ "status": "error",
+ "trace": "java.lang.NoClassDefFoundError\n\tat com.yahoo.vespa.test.samples.SampleTest.error(SampleTest.java:87)\n"
+ }
+ ]
+ },
+ {
+ "type": "test",
+ "name": "failing()",
+ "status": "failed",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "output",
+ "children": [
+ {
+ "message": "spam",
+ "at": 0,
+ "level": "info"
+ },
+ {
+ "message": "com.yahoo.vespa.test.samples.SampleTest: I have a bad feeling about this",
+ "at": 0,
+ "level": "info"
+ }
+ ]
+ },
+ {
+ "type": "failure",
+ "status": "failed",
+ "trace": "org.opentest4j.AssertionFailedError: baz ==> expected: <foo> but was: <bar>\n\tat org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:55)\n\tat org.junit.jupiter.api.AssertionUtils.failNotEqual(AssertionUtils.java:62)\n\tat org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:182)\n\tat org.junit.jupiter.api.Assertions.assertEquals(Assertions.java:1152)\n\tat com.yahoo.vespa.test.samples.SampleTest.failing(SampleTest.java:81)\n"
+ }
+ ]
+ },
+ {
+ "type": "test",
+ "name": "ignored()",
+ "status": "skipped",
+ "start": 0,
+ "end": 0
+ },
+ {
+ "type": "test",
+ "name": "inconclusive(TestReporter)",
+ "status": "inconclusive",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "output",
+ "children": [
+ {
+ "message": "spam",
+ "at": 0,
+ "level": "info"
+ },
+ {
+ "message": "I'm here with Erwin today; Erwin, what can you tell us about your cat?",
+ "at": 0,
+ "level": "info"
+ }
+ ]
+ },
+ {
+ "type": "failure",
+ "status": "inconclusive",
+ "trace": "ai.vespa.hosted.cd.InconclusiveTestException: the cat is both dead _and_ alive\n\tat com.yahoo.vespa.test.samples.SampleTest.inconclusive(SampleTest.java:93)\n"
+ }
+ ]
+ },
+ {
+ "type": "test",
+ "name": "successful()",
+ "status": "successful",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "output",
+ "children": [
+ {
+ "message": "spam",
+ "at": 0,
+ "level": "info"
+ },
+ {
+ "message": "com.yahoo.vespa.test.samples.SampleTest: <body />",
+ "at": 0,
+ "level": "info"
+ },
+ {
+ "message": "com.yahoo.vespa.test.samples.SampleTest: Very informative",
+ "at": 0,
+ "level": "info"
+ },
+ {
+ "message": "com.yahoo.vespa.test.samples.SampleTest: Oh no",
+ "at": 0,
+ "level": "warning",
+ "trace": "java.lang.IllegalArgumentException: error\n\tat com.yahoo.vespa.test.samples.SampleTest.successful(SampleTest.java:75)\nCaused by: java.lang.RuntimeException: wrapped\n\t... 1 more\n"
+ }
+ ]
+ }
+ ]
+ },
+ {
+ "type": "container",
+ "name": "Inner",
+ "status": "failed",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "container",
+ "name": "others()",
+ "status": "failed",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "output",
+ "children": [
+ {
+ "message": "spam",
+ "at": 0,
+ "level": "info"
+ }
+ ]
+ },
+ {
+ "type": "test",
+ "name": "second",
+ "status": "successful",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "output",
+ "children": [
+ {
+ "message": "Catch me if you can!",
+ "at": 0,
+ "level": "info"
+ }
+ ]
+ }
+ ]
+ },
+ {
+ "type": "test",
+ "name": "third",
+ "status": "failed",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "failure",
+ "status": "failed",
+ "trace": "org.opentest4j.AssertionFailedError: no charm\n\tat org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:39)\n\tat org.junit.jupiter.api.Assertions.fail(Assertions.java:134)\n\tat com.yahoo.vespa.test.samples.SampleTest$Inner.lambda$others$1(SampleTest.java:105)\n"
+ }
+ ]
+ }
+ ]
+ },
+ {
+ "type": "test",
+ "name": "first()",
+ "status": "successful",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "output",
+ "children": [
+ {
+ "message": "spam",
+ "at": 0,
+ "level": "info"
+ }
+ ]
+ }
+ ]
+ }
+ ]
+ },
+ {
+ "type": "container",
+ "name": "Skipped",
+ "status": "skipped",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "test",
+ "name": "disabled()",
+ "status": "skipped",
+ "start": 0,
+ "end": 0
+ }
+ ]
+ }
+ ]
+ }
+ ]
+ },
+ {
+ "type": "container",
+ "name": "JUnit Jupiter",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "container",
+ "name": "FailingTestAndBothAftersTest",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "test",
+ "name": "test()",
+ "status": "failed",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "failure",
+ "status": "failed",
+ "trace": "org.opentest4j.AssertionFailedError\n\tat org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:35)\n\tat org.junit.jupiter.api.Assertions.fail(Assertions.java:115)\n\tat com.yahoo.vespa.test.samples.FailingTestAndBothAftersTest.test(FailingTestAndBothAftersTest.java:19)\n\tSuppressed: java.lang.RuntimeException\n\t\tat com.yahoo.vespa.test.samples.FailingTestAndBothAftersTest.moreFail(FailingTestAndBothAftersTest.java:16)\n"
+ }
+ ]
+ },
+ {
+ "type": "failure",
+ "status": "error",
+ "trace": "java.lang.RuntimeException\n\tat com.yahoo.vespa.test.samples.FailingTestAndBothAftersTest.fail(FailingTestAndBothAftersTest.java:13)\n"
+ }
+ ]
+ },
+ {
+ "type": "container",
+ "name": "WrongBeforeAllTest",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "failure",
+ "status": "error",
+ "trace": "org.junit.platform.commons.JUnitException: @BeforeAll method 'void com.yahoo.vespa.test.samples.WrongBeforeAllTest.wrong()' must be static unless the test class is annotated with @TestInstance(Lifecycle.PER_CLASS).\n"
+ },
+ {
+ "type": "test",
+ "name": "fest()",
+ "status": "skipped",
+ "start": 0,
+ "end": 0
+ },
+ {
+ "type": "test",
+ "name": "test()",
+ "status": "skipped",
+ "start": 0,
+ "end": 0
+ }
+ ]
+ },
+ {
+ "type": "container",
+ "name": "FailingExtensionTest",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "test",
+ "name": "test()",
+ "status": "error",
+ "start": 0,
+ "end": 0,
+ "children": [
+ {
+ "type": "failure",
+ "status": "error",
+ "trace": "java.lang.NullPointerException\n\tat com.yahoo.vespa.test.samples.FailingExtensionTest$FailingExtension.<init>(FailingExtensionTest.java:19)\n"
+ }
+ ]
+ }
+ ]
+ }
+ ]
+ },
+ {
+ "type": "failure",
+ "status": "error",
+ "trace": "java.lang.ClassNotFoundException: School's out all summer!\n"
+ }
+ ]
+ },
+ "summary": {
+ "success": 3,
+ "failed": 5,
+ "ignored": 4,
+ "aborted": 1,
+ "inconclusive": 1,
+ "failures": [
+ {
+ "testName": "error()",
+ "testError": null,
+ "exception": "java.lang.NoClassDefFoundError\n\tat com.yahoo.vespa.test.samples.SampleTest.error(SampleTest.java:87)\n"
+ },
+ {
+ "testName": "failing()",
+ "testError": "baz ==> expected: <foo> but was: <bar>",
+ "exception": "org.opentest4j.AssertionFailedError: baz ==> expected: <foo> but was: <bar>\n\tat org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:55)\n\tat org.junit.jupiter.api.AssertionUtils.failNotEqual(AssertionUtils.java:62)\n\tat org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:182)\n\tat org.junit.jupiter.api.Assertions.assertEquals(Assertions.java:1152)\n\tat com.yahoo.vespa.test.samples.SampleTest.failing(SampleTest.java:81)\n"
+ },
+ {
+ "testName": "inconclusive(TestReporter)",
+ "testError": "the cat is both dead _and_ alive",
+ "exception": "ai.vespa.hosted.cd.InconclusiveTestException: the cat is both dead _and_ alive\n\tat com.yahoo.vespa.test.samples.SampleTest.inconclusive(SampleTest.java:93)\n"
+ },
+ {
+ "testName": "third",
+ "testError": "no charm",
+ "exception": "org.opentest4j.AssertionFailedError: no charm\n\tat org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:39)\n\tat org.junit.jupiter.api.Assertions.fail(Assertions.java:134)\n\tat com.yahoo.vespa.test.samples.SampleTest$Inner.lambda$others$1(SampleTest.java:105)\n"
+ },
+ {
+ "testName": "test()",
+ "testError": "",
+ "exception": "org.opentest4j.AssertionFailedError\n\tat org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:35)\n\tat org.junit.jupiter.api.Assertions.fail(Assertions.java:115)\n\tat com.yahoo.vespa.test.samples.FailingTestAndBothAftersTest.test(FailingTestAndBothAftersTest.java:19)\n\tSuppressed: java.lang.RuntimeException\n\t\tat com.yahoo.vespa.test.samples.FailingTestAndBothAftersTest.moreFail(FailingTestAndBothAftersTest.java:16)\n"
+ },
+ {
+ "testName": "FailingTestAndBothAftersTest",
+ "testError": null,
+ "exception": "java.lang.RuntimeException\n\tat com.yahoo.vespa.test.samples.FailingTestAndBothAftersTest.fail(FailingTestAndBothAftersTest.java:13)\n"
+ },
+ {
+ "testName": "WrongBeforeAllTest",
+ "testError": "@BeforeAll method 'void com.yahoo.vespa.test.samples.WrongBeforeAllTest.wrong()' must be static unless the test class is annotated with @TestInstance(Lifecycle.PER_CLASS).",
+ "exception": "org.junit.platform.commons.JUnitException: @BeforeAll method 'void com.yahoo.vespa.test.samples.WrongBeforeAllTest.wrong()' must be static unless the test class is annotated with @TestInstance(Lifecycle.PER_CLASS).\n"
+ },
+ {
+ "testName": "test()",
+ "testError": null,
+ "exception": "java.lang.NullPointerException\n\tat com.yahoo.vespa.test.samples.FailingExtensionTest$FailingExtension.<init>(FailingExtensionTest.java:19)\n"
+ },
+ {
+ "testName": "Production test",
+ "testError": "School's out all summer!",
+ "exception": "java.lang.ClassNotFoundException: School's out all summer!\n"
+ }
+ ]
+ },
+ "output": [
+ "00:00:00.000 spam",
+ "00:00:00.000 spam",
+ "00:00:00.000 spam",
+ "00:00:00.000 I have a bad feeling about this",
+ "00:00:00.000 spam",
+ "00:00:00.000 I'm here with Erwin today; Erwin, what can you tell us about your cat?",
+ "00:00:00.000 spam",
+ "00:00:00.000 <body />",
+ "00:00:00.000 Very informative",
+ "00:00:00.000 Oh no",
+ "00:00:00.000 spam",
+ "00:00:00.000 Catch me if you can!",
+ "00:00:00.000 spam"
+ ]
+}