You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/04/28 17:47:52 UTC

[GitHub] [geode] jdeppe-pivotal opened a new pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

jdeppe-pivotal opened a new pull request #6385:
URL: https://github.com/apache/geode/pull/6385


   Something to note when doing SSL testing: testcontainers does not let
   you set the container name (using `container_name` in your compose.yml).
   This ultimately means that reverse IP lookups produce hostnames that
   look something like `project_service_index`. The problem is that these
   names are not RFC compliant as they contain underscores. This can break
   some aspects of SSL validation. I've had to work around this by renaming
   containers in various test classes.
   
   
   Thank you for submitting a contribution to Apache Geode.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   - [ ] Does `gradlew build` run cleanly?
   
   - [ ] Have you written or updated unit tests to verify your changes?
   
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   
   ### Note:
   Please ensure that once the PR is submitted, check Concourse for build issues and
   submit an update to your PR as soon as possible. If you need help, please send an
   email to dev@geode.apache.org.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] albertogpz commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
albertogpz commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r623160840



##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-netwotk}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   */

Review comment:
       Crystal clear now. Thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] Bill commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
Bill commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r623325116



##########
File path: geode-assembly/src/acceptanceTest/resources/org/apache/geode/cache/wan/docker-compose.yml
##########
@@ -17,12 +17,8 @@
 version: '3'
 services:
   locator:
-    container_name: 'locator'
     image: 'geode:develop'
     hostname: locator
-    expose:

Review comment:
       I see you've removed all the `expose` directives in the various `docker-compose.yml` files. I suppose these still work. I guess that's because your Java code is calling testcontainers API to expose the pertinent ports (to other containers).
   
   On the one hand I can see why perhaps this information could be deemed redundant (since it's also in the Java code). On the other hand, if I could have the exposed ports information in only one place, I'd prefer it in the `yml` files since these can be used from the terminal to spin up services for experimentation.
   
   Not only is it useful to have them in the `yml` file for experimentation, their presence there gives us a clear picture (at a glance) of all the exposed ports. That's useful for maintainers.
   
   Would it make sense to reinstate these `expose` directives? Could we rely on them here in the `yml` files and remove the redundant Java code?

##########
File path: geode-assembly/src/acceptanceTest/resources/org/apache/geode/client/sni/dual-server-docker-compose.yml
##########
@@ -43,11 +37,8 @@ services:
       - ./geode-config:/geode/config:ro
       - ./scripts:/geode/scripts
   server-dolores:
-    container_name: 'server-dolores'

Review comment:
       I guess these container names really were redundant since you can refer to the containers by their service name now. ok.

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+/**
+ * This class assists in managing the lifecycle of a cluster, launched via a docker-compose
+ * configuration file, for testing. For example:
+ *
+ * <pre>
+ *
+ * &#64;ClassRule
+ * public static DockerComposeRule cluster = new DockerComposeRule().Builder()
+ *     .file("/home/bob/test/docker-compose.yml")
+ *     .service("haproxy", 15223)
+ *     .build();
+ *
+ * // Get the exposed port for haproxy
+ * cluster.getExternalPortForService("haproxy", 15223);
+ * </pre>
+ *
+ * Some limitations are as follows:
+ * <ul>
+ * <li>{@code testcontainers} does not support using {@code container_name:} attributes. If you
+ * need your container to be named explicitly, use {@link DockerComposeRule#setContainerName}</li>
+ * <li>Do not use the {@code expose} directives in your yaml file. Instead use
+ * {@link DockerComposeRule.Builder#service}
+ * to expose the relevant service and port.</li>
+ * <li>For now, this rule only handles a single instance of a service.</li>
+ * </ul>
+ *
+ * @see <a href=
+ *      "https://www.testcontainers.org/modules/docker_compose/">https://www.testcontainers.org/modules/docker_compose/</a>
+ */
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+
+    // Docker compose does not work on windows in CI. Ignore this test on windows using a
+    // RuleChain to make sure we ignore the test before the rule comes into play.
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-network}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   *
+   * @param serviceName the service who's container name to change
+   * @param newName the new container name
+   *
+   * @throws IllegalArgumentException if the service cannot be found
+   */
+  public void setContainerName(String serviceName, String newName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+    dockerClient.renameContainerCmd(containerId).withName(newName).exec();
+  }
+
+  /**
+   * Execute a command in a service container
+   *
+   * @return the stdout of the container if the command was successful, else the stderr
+   */
+  public String execForService(String serviceName, String... args) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+    Container.ExecResult result;
+    try {
+      result = container.execInContainer(args);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result.getExitCode() == 0 ? result.getStdout() : result.getStderr();
+  }
+
+  /**
+   * Execute a commond in a service container, logging the output

Review comment:
       "commond" => "command"

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+/**
+ * This class assists in managing the lifecycle of a cluster, launched via a docker-compose
+ * configuration file, for testing. For example:
+ *
+ * <pre>
+ *
+ * &#64;ClassRule
+ * public static DockerComposeRule cluster = new DockerComposeRule().Builder()
+ *     .file("/home/bob/test/docker-compose.yml")
+ *     .service("haproxy", 15223)
+ *     .build();
+ *
+ * // Get the exposed port for haproxy
+ * cluster.getExternalPortForService("haproxy", 15223);
+ * </pre>
+ *
+ * Some limitations are as follows:
+ * <ul>
+ * <li>{@code testcontainers} does not support using {@code container_name:} attributes. If you
+ * need your container to be named explicitly, use {@link DockerComposeRule#setContainerName}</li>
+ * <li>Do not use the {@code expose} directives in your yaml file. Instead use
+ * {@link DockerComposeRule.Builder#service}
+ * to expose the relevant service and port.</li>
+ * <li>For now, this rule only handles a single instance of a service.</li>
+ * </ul>
+ *
+ * @see <a href=
+ *      "https://www.testcontainers.org/modules/docker_compose/">https://www.testcontainers.org/modules/docker_compose/</a>
+ */
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+
+    // Docker compose does not work on windows in CI. Ignore this test on windows using a
+    // RuleChain to make sure we ignore the test before the rule comes into play.
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-network}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   *
+   * @param serviceName the service who's container name to change
+   * @param newName the new container name
+   *
+   * @throws IllegalArgumentException if the service cannot be found
+   */
+  public void setContainerName(String serviceName, String newName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+    dockerClient.renameContainerCmd(containerId).withName(newName).exec();
+  }
+
+  /**
+   * Execute a command in a service container
+   *
+   * @return the stdout of the container if the command was successful, else the stderr
+   */
+  public String execForService(String serviceName, String... args) {

Review comment:
       Suggest renaming `args` to `command` for descriptiveness and to conform to next method.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] bschuchardt commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
bschuchardt commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r622521160



##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-netwotk}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   */
+  public void normalizeContainerName(String serviceName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+    dockerClient.renameContainerCmd(containerId).withName(serviceName).exec();
+  }
+
+  public String execForService(String serviceName, String... args) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+    Container.ExecResult result;
+    try {
+      result = container.execInContainer(args);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result.getExitCode() == 0 ? result.getStdout() : result.getStderr();
+  }
+
+  public Long loggingExecForService(String serviceName, String... command) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+    String containerName = container.getContainerInfo().getName();
+
+    logger.info("{}: Running 'exec' command: {}", containerName, command);
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+
+    final ExecCreateCmdResponse execCreateCmdResponse = dockerClient.execCreateCmd(containerId)
+        .withAttachStdout(true).withAttachStderr(true).withCmd(command).exec();
+
+    final ToLogConsumer stdoutConsumer = new ToLogConsumer(serviceName, logger);
+    final ToLogConsumer stderrConsumer = new ToLogConsumer(serviceName, logger);
+
+    FrameConsumerResultCallback callback = new FrameConsumerResultCallback();
+    callback.addConsumer(OutputFrame.OutputType.STDOUT, stdoutConsumer);
+    callback.addConsumer(OutputFrame.OutputType.STDERR, stderrConsumer);
+
+    try {
+      dockerClient.execStartCmd(execCreateCmdResponse.getId()).exec(callback).awaitCompletion();
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    return dockerClient.inspectExecCmd(execCreateCmdResponse.getId()).exec().getExitCodeLong();
+  }
+
+
+  public Integer getExternalPortForService(String serviceName, int port) {

Review comment:
       It would be nice to have javadocs for the methods in 
   DockerComposeRule




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] albertogpz commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
albertogpz commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r622858859



##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-netwotk}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   */

Review comment:
       It is not clear to me with this description what you are doing in the method. Looking at the code, it seems you are only removing the "_1" from the container name but according to the comment, there might be other underscores in the name.
   Could you please clarify it?

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/wan/SeveralGatewayReceiversWithSamePortAndHostnameForSendersTest.java
##########
@@ -92,9 +85,11 @@
   // Docker compose does not work on windows in CI. Ignore this test on windows

Review comment:
       Shouldn't these comments be moved to DockerComposeRule.java?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r623380569



##########
File path: geode-assembly/src/acceptanceTest/resources/org/apache/geode/cache/wan/docker-compose.yml
##########
@@ -17,12 +17,8 @@
 version: '3'
 services:
   locator:
-    container_name: 'locator'
     image: 'geode:develop'
     hostname: locator
-    expose:

Review comment:
       Totally agree, but I couldn't get _testcontainers_ to work properly with the `expose` directives left in place. Possibly I'm doing something wrong. It does do something a bit weird in that it spins up an _ambassador_ container which proxies the connections to the actual containers. I think that may be part of the need to not expose the ports, but I'm not sure. More details here: https://www.testcontainers.org/modules/docker_compose/#example




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal merged pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal merged pull request #6385:
URL: https://github.com/apache/geode/pull/6385


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r623381149



##########
File path: geode-assembly/src/acceptanceTest/resources/org/apache/geode/client/sni/dual-server-docker-compose.yml
##########
@@ -43,11 +37,8 @@ services:
       - ./geode-config:/geode/config:ro
       - ./scripts:/geode/scripts
   server-dolores:
-    container_name: 'server-dolores'

Review comment:
       And _testcontainers_ errors out if they are left in place...




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r623149813



##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-netwotk}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   */
+  public void normalizeContainerName(String serviceName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+    dockerClient.renameContainerCmd(containerId).withName(serviceName).exec();
+  }
+
+  public String execForService(String serviceName, String... args) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+    Container.ExecResult result;
+    try {
+      result = container.execInContainer(args);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result.getExitCode() == 0 ? result.getStdout() : result.getStderr();
+  }
+
+  public Long loggingExecForService(String serviceName, String... command) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+    String containerName = container.getContainerInfo().getName();
+
+    logger.info("{}: Running 'exec' command: {}", containerName, command);
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+
+    final ExecCreateCmdResponse execCreateCmdResponse = dockerClient.execCreateCmd(containerId)
+        .withAttachStdout(true).withAttachStderr(true).withCmd(command).exec();
+
+    final ToLogConsumer stdoutConsumer = new ToLogConsumer(serviceName, logger);
+    final ToLogConsumer stderrConsumer = new ToLogConsumer(serviceName, logger);
+
+    FrameConsumerResultCallback callback = new FrameConsumerResultCallback();
+    callback.addConsumer(OutputFrame.OutputType.STDOUT, stdoutConsumer);
+    callback.addConsumer(OutputFrame.OutputType.STDERR, stderrConsumer);
+
+    try {
+      dockerClient.execStartCmd(execCreateCmdResponse.getId()).exec(callback).awaitCompletion();
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    return dockerClient.inspectExecCmd(execCreateCmdResponse.getId()).exec().getExitCodeLong();
+  }
+
+
+  public Integer getExternalPortForService(String serviceName, int port) {

Review comment:
       Done - thanks for the prompt!

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/wan/SeveralGatewayReceiversWithSamePortAndHostnameForSendersTest.java
##########
@@ -92,9 +85,11 @@
   // Docker compose does not work on windows in CI. Ignore this test on windows

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] Bill commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
Bill commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r624207503



##########
File path: geode-assembly/src/acceptanceTest/resources/org/apache/geode/cache/wan/docker-compose.yml
##########
@@ -17,12 +17,8 @@
 version: '3'
 services:
   locator:
-    container_name: 'locator'
     image: 'geode:develop'
     hostname: locator
-    expose:

Review comment:
       Apparently `expose` (in a docker compose file) isn't really needed:
   
   https://stackoverflow.com/questions/65784039/why-a-service-configured-with-only-expose-is-able-to-communicate-with-internet/65785558#65785558
   
   At best is was mere documentation. So I'm ok w/ removing these.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r623158509



##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-netwotk}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   */

Review comment:
       I've renamed the method to `setContainerName` to make it a bit more explicit. I had to use this in `DualServerSNIAcceptanceTest`. Without this it was producing SSL exceptions indicating that a hostname was malformed.
   
   When `docker-compose` creates containers the names end up looking something like this: `composeowub3t_locator-maeve_1`. The original 'compose' library we're replacing allowed for `container_name:` attributes in the compose yaml, but _testcontainers_ does not. So if a container does a reverse DNS lookup it gets back a hostname that is actually not RFC compliant because it contains underscores:
   ```
   root@server-dolores:/# dig -x 172.22.0.5
   
   ; <<>> DiG 9.10.3-P4-Debian <<>> -x 172.22.0.5
   ;; global options: +cmd
   ;; Got answer:
   ;; ->>HEADER<<- opcode: QUERY, status: NOERROR, id: 17690
   ;; flags: qr rd ra; QUERY: 1, ANSWER: 1, AUTHORITY: 0, ADDITIONAL: 0
   
   ;; QUESTION SECTION:
   ;5.0.22.172.in-addr.arpa.	IN	PTR
   
   ;; ANSWER SECTION:
   5.0.22.172.in-addr.arpa. 600	IN	PTR	composeowub3t_locator-maeve_1.geode-sni-test.
   ```
   Unfortunately, I don't have the stack trace, but this hostname results in an exception being thrown and SSL cannot establish a connection.
   
   Ultimately, this method is here to rename the container exactly as would happen if _testcontainers_ could handle `container_name:`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6385: GEODE-9156: Replace docker-compose-rule with testcontainers in geode-assembly

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6385:
URL: https://github.com/apache/geode/pull/6385#discussion_r623378021



##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+/**
+ * This class assists in managing the lifecycle of a cluster, launched via a docker-compose
+ * configuration file, for testing. For example:
+ *
+ * <pre>
+ *
+ * &#64;ClassRule
+ * public static DockerComposeRule cluster = new DockerComposeRule().Builder()
+ *     .file("/home/bob/test/docker-compose.yml")
+ *     .service("haproxy", 15223)
+ *     .build();
+ *
+ * // Get the exposed port for haproxy
+ * cluster.getExternalPortForService("haproxy", 15223);
+ * </pre>
+ *
+ * Some limitations are as follows:
+ * <ul>
+ * <li>{@code testcontainers} does not support using {@code container_name:} attributes. If you
+ * need your container to be named explicitly, use {@link DockerComposeRule#setContainerName}</li>
+ * <li>Do not use the {@code expose} directives in your yaml file. Instead use
+ * {@link DockerComposeRule.Builder#service}
+ * to expose the relevant service and port.</li>
+ * <li>For now, this rule only handles a single instance of a service.</li>
+ * </ul>
+ *
+ * @see <a href=
+ *      "https://www.testcontainers.org/modules/docker_compose/">https://www.testcontainers.org/modules/docker_compose/</a>
+ */
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+
+    // Docker compose does not work on windows in CI. Ignore this test on windows using a
+    // RuleChain to make sure we ignore the test before the rule comes into play.
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-network}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   *
+   * @param serviceName the service who's container name to change
+   * @param newName the new container name
+   *
+   * @throws IllegalArgumentException if the service cannot be found
+   */
+  public void setContainerName(String serviceName, String newName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+    dockerClient.renameContainerCmd(containerId).withName(newName).exec();
+  }
+
+  /**
+   * Execute a command in a service container
+   *
+   * @return the stdout of the container if the command was successful, else the stderr
+   */
+  public String execForService(String serviceName, String... args) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+    Container.ExecResult result;
+    try {
+      result = container.execInContainer(args);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result.getExitCode() == 0 ? result.getStdout() : result.getStderr();
+  }
+
+  /**
+   * Execute a commond in a service container, logging the output

Review comment:
       Fixed.

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
##########
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+/**
+ * This class assists in managing the lifecycle of a cluster, launched via a docker-compose
+ * configuration file, for testing. For example:
+ *
+ * <pre>
+ *
+ * &#64;ClassRule
+ * public static DockerComposeRule cluster = new DockerComposeRule().Builder()
+ *     .file("/home/bob/test/docker-compose.yml")
+ *     .service("haproxy", 15223)
+ *     .build();
+ *
+ * // Get the exposed port for haproxy
+ * cluster.getExternalPortForService("haproxy", 15223);
+ * </pre>
+ *
+ * Some limitations are as follows:
+ * <ul>
+ * <li>{@code testcontainers} does not support using {@code container_name:} attributes. If you
+ * need your container to be named explicitly, use {@link DockerComposeRule#setContainerName}</li>
+ * <li>Do not use the {@code expose} directives in your yaml file. Instead use
+ * {@link DockerComposeRule.Builder#service}
+ * to expose the relevant service and port.</li>
+ * <li>For now, this rule only handles a single instance of a service.</li>
+ * </ul>
+ *
+ * @see <a href=
+ *      "https://www.testcontainers.org/modules/docker_compose/">https://www.testcontainers.org/modules/docker_compose/</a>
+ */
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+
+    // Docker compose does not work on windows in CI. Ignore this test on windows using a
+    // RuleChain to make sure we ignore the test before the rule comes into play.
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-network}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   *
+   * @param serviceName the service who's container name to change
+   * @param newName the new container name
+   *
+   * @throws IllegalArgumentException if the service cannot be found
+   */
+  public void setContainerName(String serviceName, String newName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+    dockerClient.renameContainerCmd(containerId).withName(newName).exec();
+  }
+
+  /**
+   * Execute a command in a service container
+   *
+   * @return the stdout of the container if the command was successful, else the stderr
+   */
+  public String execForService(String serviceName, String... args) {

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org