Skip to content

Commit

Permalink
Improve product tests environment startup
Browse files Browse the repository at this point in the history
  • Loading branch information
wendigo authored and losipiuk committed Sep 14, 2020
1 parent 7ee94c9 commit 25cff68
Show file tree
Hide file tree
Showing 5 changed files with 327 additions and 82 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -28,19 +28,17 @@
import io.prestosql.tests.product.launcher.env.Environments;
import io.prestosql.tests.product.launcher.env.common.Standard;
import org.testcontainers.DockerClientFactory;
import org.testcontainers.containers.Container;
import org.testcontainers.containers.ContainerState;
import picocli.CommandLine.Command;

import javax.inject.Inject;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.nio.file.Path;
import java.util.Collection;
import java.util.Optional;

import static io.prestosql.tests.product.launcher.cli.Commands.runCommand;
import static io.prestosql.tests.product.launcher.env.EnvironmentListener.loggingListener;
import static java.util.Objects.requireNonNull;
import static picocli.CommandLine.Mixin;
import static picocli.CommandLine.Option;
Expand Down Expand Up @@ -144,17 +142,15 @@ public void run()
Optional<Path> environmentLogPath = logsDirBase.map(dir -> dir.resolve(environment));
environmentLogPath.ifPresent(builder::exposeLogsInHostPath);

Environment environment = builder.build();
log.info("Starting the environment '%s'", this.environment);
Environment environment = builder.build(loggingListener());
environment.start();
log.info("Environment '%s' started", this.environment);

if (background) {
killContainersReaperContainer();
return;
}

wait(environment.getContainers());
environment.awaitContainersStopped();
log.info("Exiting, the containers will exit too");
}

Expand All @@ -168,19 +164,5 @@ private void killContainersReaperContainer()
throw new UncheckedIOException(e);
}
}

private void wait(Collection<Container<?>> containers)
{
try {
while (containers.stream().anyMatch(ContainerState::isRunning)) {
Thread.sleep(1_000);
}
throw new RuntimeException("All containers have been stopped");
}
catch (InterruptedException e) {
log.info("Interrupted");
// It's OK not to restore interrupt flag here. When we return we're exiting the process.
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,6 @@
*/
package io.prestosql.tests.product.launcher.cli;

import com.github.dockerjava.api.command.InspectContainerResponse;
import com.github.dockerjava.api.command.InspectContainerResponse.ContainerState;
import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;
import com.google.inject.Module;
Expand All @@ -27,11 +25,8 @@
import io.prestosql.tests.product.launcher.env.EnvironmentFactory;
import io.prestosql.tests.product.launcher.env.EnvironmentModule;
import io.prestosql.tests.product.launcher.env.EnvironmentOptions;
import io.prestosql.tests.product.launcher.env.Environments;
import io.prestosql.tests.product.launcher.env.common.Standard;
import io.prestosql.tests.product.launcher.testcontainers.ExistingNetwork;
import net.jodah.failsafe.Failsafe;
import net.jodah.failsafe.RetryPolicy;
import org.testcontainers.containers.Container;
import org.testcontainers.containers.wait.strategy.LogMessageWaitStrategy;
import picocli.CommandLine.Mixin;
Expand All @@ -45,12 +40,11 @@
import java.util.List;
import java.util.Optional;

import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Strings.isNullOrEmpty;
import static com.google.common.base.Throwables.getStackTraceAsString;
import static io.prestosql.tests.product.launcher.cli.Commands.runCommand;
import static io.prestosql.tests.product.launcher.docker.ContainerUtil.exposePort;
import static io.prestosql.tests.product.launcher.env.DockerContainer.cleanOrCreateHostPath;
import static io.prestosql.tests.product.launcher.env.EnvironmentListener.loggingListener;
import static io.prestosql.tests.product.launcher.env.common.Standard.CONTAINER_TEMPTO_PROFILE_CONFIG;
import static java.util.Objects.requireNonNull;
import static org.testcontainers.containers.BindMode.READ_ONLY;
Expand Down Expand Up @@ -160,18 +154,12 @@ public Execution(EnvironmentFactory environmentFactory, EnvironmentOptions envir
@Override
public void run()
{
RetryPolicy<Object> retryPolicy = new RetryPolicy<>()
.withMaxRetries(startupRetries)
.onFailedAttempt(event -> log.warn("Could not start environment '%s': %s", environment, getStackTraceAsString(event.getLastFailure())))
.onRetry(event -> log.info("Trying to start environment '%s', %d failed attempt(s)", environment, event.getAttemptCount() + 1))
.onSuccess(event -> log.info("Environment '%s' started in %s, %d attempt(s)", environment, event.getElapsedTime(), event.getAttemptCount()))
.onFailure(event -> log.info("Environment '%s' failed to start in attempt(s): %d: %s", environment, event.getAttemptCount(), event.getFailure()));

try (UncheckedCloseable ignore = this::cleanUp) {
Environment environment = Failsafe.with(retryPolicy)
.get(() -> tryStartEnvironment());

awaitTestsCompletion(environment);
try (Environment environment = startEnvironment()) {
long exitCode = environment.awaitTestsCompletion();

if (exitCode != 0L) {
throw new RuntimeException("Tests exited with " + exitCode);
}
}
catch (Throwable e) {
// log failure (tersely) because cleanup may take some time
Expand All @@ -180,16 +168,13 @@ public void run()
}
}

private Environment tryStartEnvironment()
private Environment startEnvironment()
{
Environment environment = getEnvironment();

if (!attach) {
log.info("Pruning old environment(s)");
Environments.pruneEnvironment();

log.info("Starting the environment '%s' with configuration %s", this.environment, environmentConfig);
environment.start();
environment.start(startupRetries);
}
else {
DockerContainer tests = (DockerContainer) environment.getContainer("tests");
Expand All @@ -202,14 +187,6 @@ private Environment tryStartEnvironment()
return environment;
}

private void cleanUp()
{
if (!attach) {
log.info("Done, cleaning up");
Environments.pruneEnvironment();
}
}

private Environment getEnvironment()
{
Environment.Builder environment = environmentFactory.get(this.environment)
Expand Down Expand Up @@ -263,8 +240,9 @@ private Environment getEnvironment()
environmentConfig.extendEnvironment(this.environment).ifPresent(extender -> extender.extendEnvironment(environment));

logsDirBase.ifPresent(environment::exposeLogsInHostPath);

return environment
.build();
.build(loggingListener());
}

private static Iterable<? extends String> reportsDirOptions(Path path)
Expand All @@ -286,31 +264,6 @@ private void mountReportsDir(Container container)
container.withFileSystemBind(reportsDirBase.toString(), CONTAINER_REPORTS_DIR, READ_WRITE);
log.info("Exposing tests report dir in host directory '%s'", reportsDirBase);
}

private void awaitTestsCompletion(Environment environment)
{
Container<?> container = environment.getContainer("tests");

log.info("Waiting for test completion");
try {
while (container.isRunning()) {
Thread.sleep(1000);
}

InspectContainerResponse containerInfo = container.getCurrentContainerInfo();
ContainerState containerState = containerInfo.getState();
Long exitCode = containerState.getExitCodeLong();
log.info("Test container %s is %s, with exitCode %s", containerInfo.getId(), containerState.getStatus(), exitCode);
checkState(exitCode != null, "No exitCode for tests container %s in state %s", container, containerState);
if (exitCode != 0L) {
throw new RuntimeException("Tests exited with " + exitCode);
}
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException("Interrupted", e);
}
}
}

private interface UncheckedCloseable
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package io.prestosql.tests.product.launcher.env;

import com.github.dockerjava.api.command.InspectContainerResponse;
import com.google.common.base.Stopwatch;
import com.google.common.io.RecursiveDeleteOption;
import io.airlift.log.Logger;
Expand All @@ -31,7 +32,9 @@
import java.util.Arrays;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Optional;

import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.io.MoreFiles.deleteRecursively;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
Expand All @@ -42,6 +45,7 @@ public class DockerContainer
{
private static final Logger log = Logger.get(DockerContainer.class);
private List<String> logPaths = new ArrayList<>();
private Optional<EnvironmentListener> listener = Optional.empty();

public DockerContainer(String dockerImageName)
{
Expand All @@ -51,6 +55,12 @@ public DockerContainer(String dockerImageName)
setCopyToFileContainerPathMap(new LinkedHashMap<>());
}

public DockerContainer withEnvironmentListener(Optional<EnvironmentListener> listener)
{
this.listener = requireNonNull(listener, "listener is null");
return this;
}

@Override
public void addFileSystemBind(String hostPath, String containerPath, BindMode mode)
{
Expand Down Expand Up @@ -110,11 +120,39 @@ public void exposeLogsInHostPath(Path hostBasePath)
logPaths = null;
}

@Override
protected void containerIsStarting(InspectContainerResponse containerInfo)
{
super.containerIsStarting(containerInfo);
this.listener.ifPresent(listener -> listener.containerStarting(this, containerInfo));
}

@Override
protected void containerIsStarted(InspectContainerResponse containerInfo)
{
super.containerIsStarted(containerInfo);
this.listener.ifPresent(listener -> listener.containerStarted(this, containerInfo));
}

@Override
protected void containerIsStopping(InspectContainerResponse containerInfo)
{
super.containerIsStopping(containerInfo);
this.listener.ifPresent(listener -> listener.containerStopping(this, containerInfo));
}

@Override
protected void containerIsStopped(InspectContainerResponse containerInfo)
{
super.containerIsStopped(containerInfo);
this.listener.ifPresent(listener -> listener.containerStopped(this, containerInfo));
}

private void copyFileToContainer(String containerPath, Runnable copy)
{
Stopwatch stopwatch = Stopwatch.createStarted();
copy.run();
log.info("Copied files into %s in %.1f s", containerPath, stopwatch.elapsed(MILLISECONDS) / 1000.);
log.info("Copied files into %s %s in %.1f s", this, containerPath, stopwatch.elapsed(MILLISECONDS) / 1000.);
}

// Mounting a non-existing file results in docker creating a directory. This is often not the desired effect. Fail fast instead.
Expand All @@ -130,6 +168,15 @@ public void clearDependencies()
dependencies.clear();
}

@Override
public String toString()
{
return toStringHelper(this)
.add("dockerImageName", getDockerImageName())
.add("networkAliases", getNetworkAliases())
.toString();
}

public static void cleanOrCreateHostPath(Path path)
{
try {
Expand Down
Loading

0 comments on commit 25cff68

Please sign in to comment.