Skip to content

Commit

Permalink
Skip product tests when impacted features doesn't match test environment
Browse files Browse the repository at this point in the history
  • Loading branch information
nineinchnick authored and electrum committed Apr 12, 2022
1 parent 385161b commit 6a69592
Show file tree
Hide file tree
Showing 48 changed files with 516 additions and 314 deletions.
21 changes: 21 additions & 0 deletions testing/trino-product-tests-launcher/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,17 @@
<main-class>io.trino.tests.product.launcher.cli.Launcher</main-class>
</properties>

<dependencyManagement>
<dependencies>
<!-- Trino -->
<dependency>
<groupId>org.yaml</groupId>
<artifactId>snakeyaml</artifactId>
<version>1.28</version>
</dependency>
</dependencies>
</dependencyManagement>

<dependencies>
<dependency>
<groupId>io.trino</groupId>
Expand All @@ -43,6 +54,16 @@
<artifactId>units</artifactId>
</dependency>

<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>

<dependency>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-yaml</artifactId>
</dependency>

<dependency>
<groupId>com.github.docker-java</groupId>
<artifactId>docker-java-api</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.Future;
Expand All @@ -59,6 +60,7 @@
import static io.airlift.units.Duration.succinctNanos;
import static io.trino.tests.product.launcher.cli.Commands.runCommand;
import static io.trino.tests.product.launcher.cli.SuiteRun.TestRunResult.HEADER;
import static io.trino.tests.product.launcher.cli.TestRun.Execution.ENVIRONMENT_SKIPPED_EXIT_CODE;
import static java.lang.Math.max;
import static java.lang.String.format;
import static java.lang.management.ManagementFactory.getThreadMXBean;
Expand Down Expand Up @@ -124,6 +126,9 @@ public static class SuiteRunOptions
@Option(names = "--cli-executable", paramLabel = "<jar>", description = "Path to CLI executable " + DEFAULT_VALUE, defaultValue = "${cli.bin}")
public File cliJar;

@Option(names = "--impacted-features", paramLabel = "<file>", description = "Only run tests in environments with these features " + DEFAULT_VALUE)
public Optional<File> impactedFeatures;

@Option(names = "--logs-dir", paramLabel = "<dir>", description = "Location of the exported logs directory " + DEFAULT_VALUE)
public Optional<Path> logsDirBase;

Expand Down Expand Up @@ -266,40 +271,41 @@ public TestRunResult executeSuiteTestRun(String suiteName, SuiteTestRun suiteTes
suiteTestRun,
environmentConfig,
new Duration(0, MILLISECONDS),
OptionalInt.empty(),
Optional.of(new Exception("Test execution not attempted because suite total running time limit was exhausted")));
}

log.info("Starting test run %s with config %s and remaining timeout %s", suiteTestRun, environmentConfig, testRunOptions.timeout);
log.info("Execute this test run using:\n%s test run %s", environmentOptions.launcherBin, OptionsPrinter.format(environmentOptions, testRunOptions));

Stopwatch stopwatch = Stopwatch.createStarted();
Optional<Throwable> exception = runTest(runId, environmentConfig, testRunOptions);
return new TestRunResult(suiteName, runId, suiteTestRun, environmentConfig, succinctNanos(stopwatch.stop().elapsed(NANOSECONDS)), exception);
try {
int exitCode = runTest(runId, environmentConfig, testRunOptions);
Optional<Throwable> exception = Optional.empty();
if (exitCode != 0 && exitCode != ENVIRONMENT_SKIPPED_EXIT_CODE) {
exception = Optional.of(new RuntimeException(format("Tests exited with code %d", exitCode)));
}
return new TestRunResult(suiteName, runId, suiteTestRun, environmentConfig, succinctNanos(stopwatch.stop().elapsed(NANOSECONDS)), OptionalInt.of(exitCode), exception);
}
catch (RuntimeException e) {
return new TestRunResult(suiteName, runId, suiteTestRun, environmentConfig, succinctNanos(stopwatch.stop().elapsed(NANOSECONDS)), OptionalInt.empty(), Optional.of(e));
}
}

private static String generateRandomRunId()
{
return UUID.randomUUID().toString().replace("-", "");
}

private Optional<Throwable> runTest(String runId, EnvironmentConfig environmentConfig, TestRun.TestRunOptions testRunOptions)
private int runTest(String runId, EnvironmentConfig environmentConfig, TestRun.TestRunOptions testRunOptions)
{
try {
TestRun.Execution execution = new TestRun.Execution(environmentFactory, environmentOptions, environmentConfig, testRunOptions);
TestRun.Execution execution = new TestRun.Execution(environmentFactory, environmentOptions, environmentConfig, testRunOptions);

log.info("Test run %s started", runId);
int exitCode = execution.call();
log.info("Test run %s finished", runId);
log.info("Test run %s started", runId);
int exitCode = execution.call();
log.info("Test run %s finished", runId);

if (exitCode > 0) {
return Optional.of(new RuntimeException(format("Tests exited with code %d", exitCode)));
}

return Optional.empty();
}
catch (RuntimeException e) {
return Optional.of(e);
}
return exitCode;
}

private TestRun.TestRunOptions createTestRunOptions(String runId, String suiteName, SuiteTestRun suiteTestRun, EnvironmentConfig environmentConfig, Optional<Path> logsDirBase)
Expand All @@ -310,6 +316,7 @@ private TestRun.TestRunOptions createTestRunOptions(String runId, String suiteNa
testRunOptions.testArguments = suiteTestRun.getTemptoRunArguments();
testRunOptions.testJar = suiteRunOptions.testJar;
testRunOptions.cliJar = suiteRunOptions.cliJar;
testRunOptions.impactedFeatures = suiteRunOptions.impactedFeatures;
String suiteRunId = suiteRunId(runId, suiteName, suiteTestRun, environmentConfig);
testRunOptions.reportsDir = Paths.get("testing/trino-product-tests/target/reports/" + suiteRunId);
testRunOptions.logsDirBase = logsDirBase.map(dir -> dir.resolve(suiteRunId));
Expand Down Expand Up @@ -357,15 +364,17 @@ static class TestRunResult
private final EnvironmentConfig environmentConfig;
private final Duration duration;
private final Optional<Throwable> throwable;
private final OptionalInt exitCode;
private final String suiteName;

public TestRunResult(String suiteName, String runId, SuiteTestRun suiteRun, EnvironmentConfig environmentConfig, Duration duration, Optional<Throwable> throwable)
public TestRunResult(String suiteName, String runId, SuiteTestRun suiteRun, EnvironmentConfig environmentConfig, Duration duration, OptionalInt exitCode, Optional<Throwable> throwable)
{
this.suiteName = suiteName;
this.runId = runId;
this.suiteRun = requireNonNull(suiteRun, "suiteRun is null");
this.environmentConfig = requireNonNull(environmentConfig, "environmentConfig is null");
this.duration = requireNonNull(duration, "duration is null");
this.exitCode = exitCode;
this.throwable = requireNonNull(throwable, "throwable is null");
}

Expand All @@ -374,6 +383,11 @@ public boolean hasFailed()
return this.throwable.isPresent();
}

public boolean wasSkipped()
{
return this.exitCode.orElse(0) == ENVIRONMENT_SKIPPED_EXIT_CODE;
}

@Override
public String toString()
{
Expand All @@ -395,9 +409,17 @@ public Object[] toRow()
suiteRun.getEnvironmentName(),
environmentConfig.getConfigName(),
suiteRun.getExtraOptions(),
hasFailed() ? "FAILED" : "SUCCESS",
getStatusString(),
duration,
throwable.map(Throwable::getMessage).orElse("-")};
}

private String getStatusString()
{
if (wasSkipped()) {
return "SKIPPED";
}
return hasFailed() ? "FAILED" : "SUCCESS";
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.Files;
import com.google.inject.Module;
import io.airlift.log.Logger;
import io.airlift.units.Duration;
Expand All @@ -39,6 +40,8 @@
import javax.inject.Inject;

import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collection;
Expand All @@ -47,6 +50,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.Callable;
import java.util.function.Supplier;

import static com.google.common.base.Strings.isNullOrEmpty;
import static com.google.common.collect.ImmutableList.toImmutableList;
Expand All @@ -59,6 +63,9 @@
import static java.lang.StrictMath.toIntExact;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.groupingBy;
import static java.util.stream.Collectors.mapping;
import static java.util.stream.Collectors.toList;
import static org.testcontainers.containers.BindMode.READ_ONLY;
import static org.testcontainers.containers.BindMode.READ_WRITE;
import static org.testcontainers.utility.MountableFile.forClasspathResource;
Expand Down Expand Up @@ -119,6 +126,9 @@ public static class TestRunOptions
@Option(names = "--option", paramLabel = "<option>", description = "Extra options to provide to environment (property can be used multiple times; format is key=value)")
public Map<String, String> extraOptions = new HashMap<>();

@Option(names = "--impacted-features", paramLabel = "<file>", description = "Skip tests not using these features " + DEFAULT_VALUE)
public Optional<File> impactedFeatures;

@Option(names = "--attach", description = "attach to an existing environment")
public boolean attach;

Expand Down Expand Up @@ -166,6 +176,9 @@ public static class Execution
private final Optional<Path> logsDirBase;
private final EnvironmentConfig environmentConfig;
private final Map<String, String> extraOptions;
private final Optional<List<String>> impactedFeatures;

public static final Integer ENVIRONMENT_SKIPPED_EXIT_CODE = 98;

@Inject
public Execution(EnvironmentFactory environmentFactory, EnvironmentOptions environmentOptions, EnvironmentConfig environmentConfig, TestRunOptions testRunOptions)
Expand All @@ -187,6 +200,18 @@ public Execution(EnvironmentFactory environmentFactory, EnvironmentOptions envir
this.logsDirBase = requireNonNull(testRunOptions.logsDirBase, "testRunOptions.logsDirBase is empty");
this.environmentConfig = requireNonNull(environmentConfig, "environmentConfig is null");
this.extraOptions = ImmutableMap.copyOf(requireNonNull(testRunOptions.extraOptions, "testRunOptions.extraOptions is null"));
Optional<File> impactedFeaturesFile = requireNonNull(testRunOptions.impactedFeatures, "testRunOptions.impactedFeatures is null");
if (impactedFeaturesFile.isPresent()) {
try {
this.impactedFeatures = Optional.of(Files.asCharSource(impactedFeaturesFile.get(), StandardCharsets.UTF_8).readLines());
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
else {
this.impactedFeatures = Optional.empty();
}
}

@Override
Expand Down Expand Up @@ -219,19 +244,57 @@ public Integer call()

private Integer tryExecuteTests()
{
try (Environment environment = startEnvironment()) {
return toIntExact(environment.awaitTestsCompletion());
Environment environment = getEnvironment();
if (!hasImpactedFeatures(environment)) {
log.warn("Skipping test due to impacted features not overlapping with any features configured in environment");
return toIntExact(ENVIRONMENT_SKIPPED_EXIT_CODE);
}
try (Environment runningEnvironment = startEnvironment(environment)) {
return toIntExact(runningEnvironment.awaitTestsCompletion());
}
catch (RuntimeException e) {
log.warn(e, "Failed to execute tests");
return ExitCode.SOFTWARE;
}
}

private Environment startEnvironment()
private boolean hasImpactedFeatures(Environment environment)
{
Environment environment = getEnvironment();
if (impactedFeatures.isEmpty()) {
return true;
}
if (impactedFeatures.get().size() == 0) {
return false;
}
Map<String, List<String>> featuresByName = impactedFeatures.get().stream().collect(groupingBy(feature -> {
String[] parts = feature.split(":", 2);
return parts.length < 1 ? "" : parts[0];
}, mapping(feature -> {
String[] parts = feature.split(":", 2);
return parts.length < 2 ? "" : parts[1];
}, toList())));
// see PluginReader. printPluginFeatures() for all possible feature prefixes
Map<String, Supplier<List<String>>> environmentFeaturesByName = Map.of(
"connector", environment::getConfiguredConnectors,
"passwordAuthenticator", environment::getConfiguredPasswordAuthenticators);
for (Map.Entry<String, List<String>> entry : featuresByName.entrySet()) {
String name = entry.getKey();
List<String> features = entry.getValue();
if (!environmentFeaturesByName.containsKey(name)) {
return true;
}
List<String> environmentFeatures = environmentFeaturesByName.get(name).get();
log.info("Checking if impacted %s %s are overlapping with %s configured in the environment",
name, features, environmentFeatures);
if (environmentFeatures.stream().anyMatch(features::contains)) {
return true;
}
}
return false;
}

private Environment startEnvironment(Environment environment)
{
Collection<DockerContainer> allContainers = environment.getContainers();
DockerContainer testsContainer = environment.getContainer(TESTS);

Expand Down Expand Up @@ -275,7 +338,6 @@ private Environment getEnvironment()
if (System.getenv("CONTINUOUS_INTEGRATION") != null) {
container.withEnv("CONTINUOUS_INTEGRATION", "true");
}

container
// the test jar is hundreds MB and file system bind is much more efficient
.withFileSystemBind(testJar.getPath(), "/docker/test.jar", READ_ONLY)
Expand Down
Loading

0 comments on commit 6a69592

Please sign in to comment.