From 5869de27cda6cdeaab33466ed62b774824e36d7d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Dec 2023 17:19:44 +0800 Subject: [PATCH 1/6] Apple M* Chip --- connector/docker-integration-tests/pom.xml | 13 +- .../sql/jdbc/DB2KrbIntegrationSuite.scala | 15 ++- .../sql/jdbc/DockerJDBCIntegrationSuite.scala | 122 ++++++++++-------- .../sql/jdbc/MariaDBKrbIntegrationSuite.scala | 18 +-- .../jdbc/PostgresKrbIntegrationSuite.scala | 16 +-- pom.xml | 15 +-- project/SparkBuild.scala | 6 +- 7 files changed, 112 insertions(+), 93 deletions(-) diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index ac8d2990c0e6..e3c1c5131770 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -35,12 +35,6 @@ - - com.spotify - docker-client - test - shaded - org.apache.httpcomponents httpclient @@ -55,7 +49,7 @@ com.google.guava guava - 18.0 + 19.0 org.apache.spark @@ -139,5 +133,10 @@ mysql-connector-j test + + com.github.docker-java + docker-java + test + diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala index 9b518d61d252..66e2afbb6eff 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala @@ -21,7 +21,7 @@ import java.security.PrivilegedExceptionAction import java.sql.Connection import javax.security.auth.login.Configuration -import com.spotify.docker.client.messages.{ContainerConfig, HostConfig} +import com.github.dockerjava.api.model.{AccessMode, Bind, ContainerConfig, HostConfig, Volume} import org.apache.hadoop.security.{SecurityUtil, UserGroupInformation} import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS import org.scalatest.time.SpanSugar._ @@ -66,14 +66,15 @@ class DB2KrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { } override def beforeContainerStart( - hostConfigBuilder: HostConfig.Builder, - containerConfigBuilder: ContainerConfig.Builder): Unit = { + hostConfigBuilder: HostConfig, + containerConfigBuilder: ContainerConfig): Unit = { copyExecutableResource("db2_krb_setup.sh", initDbDir, replaceIp) - hostConfigBuilder.appendBinds( - HostConfig.Bind.from(initDbDir.getAbsolutePath) - .to("/var/custom").readOnly(true).build() - ) + val newBind = new Bind( + initDbDir.getAbsolutePath, + new Volume("/var/custom"), + AccessMode.ro) + hostConfigBuilder.withBinds(hostConfigBuilder.getBinds :+ newBind: _*) } } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index bcad9ae874eb..143b6041330d 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -20,14 +20,17 @@ package org.apache.spark.sql.jdbc import java.net.ServerSocket import java.sql.{Connection, DriverManager} import java.util.Properties +import java.util.concurrent.TimeUnit import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal -import com.spotify.docker.client._ -import com.spotify.docker.client.DockerClient.{ListContainersParam, LogsParam} -import com.spotify.docker.client.exceptions.ImageNotFoundException -import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding} +import com.github.dockerjava.api.DockerClient +import com.github.dockerjava.api.async.{ResultCallback, ResultCallbackTemplate} +import com.github.dockerjava.api.command.CreateContainerResponse +import com.github.dockerjava.api.exception.NotFoundException +import com.github.dockerjava.api.model._ +import com.github.dockerjava.core.DockerClientBuilder import org.scalatest.concurrent.Eventually import org.scalatest.time.SpanSugar._ @@ -88,8 +91,8 @@ abstract class DatabaseOnDocker { * Optional step before container starts */ def beforeContainerStart( - hostConfigBuilder: HostConfig.Builder, - containerConfigBuilder: ContainerConfig.Builder): Unit = {} + hostConfigBuilder: HostConfig, + containerConfigBuilder: ContainerConfig): Unit = {} } abstract class DockerJDBCIntegrationSuite @@ -111,56 +114,70 @@ abstract class DockerJDBCIntegrationSuite sock.close() port } - private var containerId: String = _ + private var container: CreateContainerResponse = _ private var pulled: Boolean = false protected var jdbcUrl: String = _ override def beforeAll(): Unit = runIfTestsEnabled(s"Prepare for ${this.getClass.getName}") { super.beforeAll() try { - docker = DefaultDockerClient.fromEnv.build() + docker = DockerClientBuilder.getInstance().build() // Check that Docker is actually up try { - docker.ping() + docker.pingCmd().exec() } catch { case NonFatal(e) => log.error("Exception while connecting to Docker. Check whether Docker is running.") throw e } - // Ensure that the Docker image is installed: try { - docker.inspectImage(db.imageName) + // Ensure that the Docker image is installed: + docker.inspectImageCmd("mariadb:10.5.12").exec() } catch { - case e: ImageNotFoundException => + case e: NotFoundException => log.warn(s"Docker image ${db.imageName} not found; pulling image from registry") - docker.pull(db.imageName) + docker.pullImageCmd(db.imageName) + .start() + .awaitCompletion(connectionTimeout.value.toSeconds, TimeUnit.SECONDS) pulled = true } - val hostConfigBuilder = HostConfig.builder() - .privileged(db.privileged) - .networkMode("bridge") - .ipcMode(if (db.usesIpc) "host" else "") - .portBindings( - Map(s"${db.jdbcPort}/tcp" -> List(PortBinding.of(dockerIp, externalPort)).asJava).asJava) - // Create the database container: - val containerConfigBuilder = ContainerConfig.builder() - .image(db.imageName) - .networkDisabled(false) - .env(db.env.map { case (k, v) => s"$k=$v" }.toSeq.asJava) - .exposedPorts(s"${db.jdbcPort}/tcp") - if (db.getEntryPoint.isDefined) { - containerConfigBuilder.entrypoint(db.getEntryPoint.get) - } - if (db.getStartupProcessName.isDefined) { - containerConfigBuilder.cmd(db.getStartupProcessName.get) + + docker.pullImageCmd(db.imageName) + .start() + .awaitCompletion(connectionTimeout.value.toSeconds, TimeUnit.SECONDS) + + val hostConfig = HostConfig + .newHostConfig() + .withNetworkMode("bridge") + .withPrivileged(db.privileged) + .withPortBindings(PortBinding.parse(s"$dockerIp:$externalPort:${db.jdbcPort}")) + + if (db.usesIpc) { + hostConfig.withIpcMode("host") } - db.beforeContainerStart(hostConfigBuilder, containerConfigBuilder) - containerConfigBuilder.hostConfig(hostConfigBuilder.build()) - val config = containerConfigBuilder.build() + + val containerConfig = new ContainerConfig() + + db.beforeContainerStart(hostConfig, containerConfig) + // Create the database container: - containerId = docker.createContainer(config).id + val createContainerCmd = docker.createContainerCmd(db.imageName) + .withHostConfig(hostConfig) + .withExposedPorts(ExposedPort.tcp(db.jdbcPort)) + .withEnv(db.env.map { case (k, v) => s"$k=$v" }.toList.asJava) + .withNetworkDisabled(false) + + + db.getEntryPoint.foreach(ep => createContainerCmd.withEntrypoint(ep)) + db.getStartupProcessName.foreach(n => createContainerCmd.withCmd(n)) + + container = createContainerCmd.exec() // Start the container and wait until the database can accept JDBC connections: - docker.startContainer(containerId) + docker.startContainerCmd(container.getId).exec() + eventually(connectionTimeout, interval(1.second)) { + val response = docker.inspectContainerCmd(container.getId).exec() + assert(response.getState.getRunning) + } jdbcUrl = db.getJdbcUrl(dockerIp, externalPort) var conn: Connection = null eventually(connectionTimeout, interval(1.second)) { @@ -206,36 +223,39 @@ abstract class DockerJDBCIntegrationSuite def dataPreparation(connection: Connection): Unit private def cleanupContainer(): Unit = { - if (docker != null && containerId != null && !keepContainer) { + if (docker != null && container != null && !keepContainer) { try { - docker.killContainer(containerId) + docker.killContainerCmd(container.getId).exec() } catch { case NonFatal(e) => - val exitContainerIds = - docker.listContainers(ListContainersParam.withStatusExited()).asScala.map(_.id()) - if (exitContainerIds.contains(containerId)) { - logWarning(s"Container $containerId already stopped") - } else { - logWarning(s"Could not stop container $containerId", e) - } + val response = docker.inspectContainerCmd(container.getId).exec() + logWarning(s"Container $container already stopped") + val status = Option(response).map(_.getState.getStatus).getOrElse("unknown") + logWarning(s"Could not stop container $container at stage '$status'", e) } finally { logContainerOutput() - docker.removeContainer(containerId) + docker.removeContainerCmd(container.getId).exec() if (removePulledImage && pulled) { - docker.removeImage(db.imageName) + docker.removeImageCmd(db.imageName).exec() } } } } private def logContainerOutput(): Unit = { - val logStream = docker.logs(containerId, LogsParam.stdout(), LogsParam.stderr()) try { - logInfo("\n\n===== CONTAINER LOGS FOR container Id: " + containerId + " =====") - logInfo(logStream.readFully()) - logInfo("\n\n===== END OF CONTAINER LOGS FOR container Id: " + containerId + " =====") + logInfo("\n\n===== CONTAINER LOGS FOR container Id: " + container + " =====") + + docker.logContainerCmd(container.getId) + .withStdOut(true) + .withStdErr(true) + .withFollowStream(true) + .withSince(0) + .exec(new ResultCallbackTemplate[ResultCallback[Frame], Frame] { + override def onNext(f: Frame): Unit = logInfo(f.toString) + }) + logInfo("\n\n===== END OF CONTAINER LOGS FOR container Id: " + container + " =====") } finally { - logStream.close() } } } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala index 873d5ad1ee43..49c9e3dba0d7 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.jdbc import javax.security.auth.login.Configuration -import com.spotify.docker.client.messages.{ContainerConfig, HostConfig} +import com.github.dockerjava.api.model.{AccessMode, Bind, ContainerConfig, HostConfig, Volume} import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnectionProvider import org.apache.spark.tags.DockerTest @@ -52,17 +52,17 @@ class MariaDBKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { Some("/docker-entrypoint/mariadb_docker_entrypoint.sh") override def beforeContainerStart( - hostConfigBuilder: HostConfig.Builder, - containerConfigBuilder: ContainerConfig.Builder): Unit = { + hostConfigBuilder: HostConfig, + containerConfigBuilder: ContainerConfig): Unit = { copyExecutableResource("mariadb_docker_entrypoint.sh", entryPointDir, replaceIp) copyExecutableResource("mariadb_krb_setup.sh", initDbDir, replaceIp) - hostConfigBuilder.appendBinds( - HostConfig.Bind.from(entryPointDir.getAbsolutePath) - .to("/docker-entrypoint").readOnly(true).build(), - HostConfig.Bind.from(initDbDir.getAbsolutePath) - .to("/docker-entrypoint-initdb.d").readOnly(true).build() - ) + val binds = + Seq(entryPointDir -> "/docker-entrypoint", initDbDir -> "/docker-entrypoint-initdb.d") + .map { case (from, to) => + new Bind(from.getAbsolutePath, new Volume(to), AccessMode.ro) + } + hostConfigBuilder.withBinds(hostConfigBuilder.getBinds ++ binds: _*) } } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala index 4debe24754de..baf24b3c1357 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.jdbc import javax.security.auth.login.Configuration -import com.spotify.docker.client.messages.{ContainerConfig, HostConfig} +import com.github.dockerjava.api.model.{AccessMode, Bind, ContainerConfig, HostConfig, Volume} import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnectionProvider import org.apache.spark.tags.DockerTest @@ -48,14 +48,14 @@ class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { s"jdbc:postgresql://$ip:$port/postgres?user=$principal&gsslib=gssapi" override def beforeContainerStart( - hostConfigBuilder: HostConfig.Builder, - containerConfigBuilder: ContainerConfig.Builder): Unit = { + hostConfigBuilder: HostConfig, + containerConfigBuilder: ContainerConfig): Unit = { copyExecutableResource("postgres_krb_setup.sh", initDbDir, replaceIp) - - hostConfigBuilder.appendBinds( - HostConfig.Bind.from(initDbDir.getAbsolutePath) - .to("/docker-entrypoint-initdb.d").readOnly(true).build() - ) + val newBind = new Bind( + initDbDir.getAbsolutePath, + new Volume("/docker-entrypoint-initdb.d"), + AccessMode.ro) + hostConfigBuilder.withBinds(hostConfigBuilder.getBinds :+ newBind: _*) } } diff --git a/pom.xml b/pom.xml index 09a1d3d49998..cb5b514909ce 100644 --- a/pom.xml +++ b/pom.xml @@ -1237,20 +1237,19 @@ test - com.spotify - docker-client - 8.14.1 + com.github.docker-java + docker-java + 3.3.4 test - shaded - - guava - com.google.guava - commons-logging commons-logging + + com.github.docker-java + docker-java-transport-netty + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index db546dcdd5bb..99c0f8037631 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -415,8 +415,7 @@ object SparkBuild extends PomBuild { /* Protobuf settings */ enable(SparkProtobuf.settings)(protobuf) - // SPARK-14738 - Remove docker tests from main Spark build - // enable(DockerIntegrationTests.settings)(dockerIntegrationTests) + enable(DockerIntegrationTests.settings)(dockerIntegrationTests) if (!profiles.contains("volcano")) { enable(Volcano.settings)(kubernetes) @@ -946,7 +945,8 @@ object Unsafe { object DockerIntegrationTests { // This serves to override the override specified in DependencyOverrides: lazy val settings = Seq( - dependencyOverrides += "com.google.guava" % "guava" % "18.0" + libraryDependencies += "com.google.guava" % "guava" % "19.0", + dependencyOverrides += "com.google.guava" % "guava" % "19.0" ) } From 05544299fb4eb386ea0a40dbb5be0e5f54caf7fd Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Dec 2023 17:56:37 +0800 Subject: [PATCH 2/6] clean --- connector/docker-integration-tests/pom.xml | 8 -------- 1 file changed, 8 deletions(-) diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index e3c1c5131770..23e8f26a99c0 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -95,14 +95,6 @@ hadoop-minikdc test - - - org.glassfish.jersey.bundles.repackaged - jersey-guava - 2.25.1 - test - org.mariadb.jdbc mariadb-java-client From 6ba25b07aa050a0108fde0c66363ef20ba5d7883 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Dec 2023 18:06:02 +0800 Subject: [PATCH 3/6] clean --- project/SparkBuild.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 99c0f8037631..763ce847b3c7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -944,10 +944,7 @@ object Unsafe { object DockerIntegrationTests { // This serves to override the override specified in DependencyOverrides: - lazy val settings = Seq( - libraryDependencies += "com.google.guava" % "guava" % "19.0", - dependencyOverrides += "com.google.guava" % "guava" % "19.0" - ) + lazy val settings = Seq(dependencyOverrides += "com.google.guava" % "guava" % "19.0") } /** From b606f659ec78ca781b1cd3639ed9e18e9cb37f05 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Dec 2023 18:24:42 +0800 Subject: [PATCH 4/6] zerode --- connector/docker-integration-tests/pom.xml | 5 +++++ .../spark/sql/jdbc/DockerJDBCIntegrationSuite.scala | 10 ++++++++-- pom.xml | 10 ++++++++++ 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index 23e8f26a99c0..4cca3ef12ae5 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -130,5 +130,10 @@ docker-java test + + com.github.docker-java + docker-java-transport-zerodep + test + diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index 143b6041330d..d7b52106f3ba 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -30,7 +30,8 @@ import com.github.dockerjava.api.async.{ResultCallback, ResultCallbackTemplate} import com.github.dockerjava.api.command.CreateContainerResponse import com.github.dockerjava.api.exception.NotFoundException import com.github.dockerjava.api.model._ -import com.github.dockerjava.core.DockerClientBuilder +import com.github.dockerjava.core.{DefaultDockerClientConfig, DockerClientImpl} +import com.github.dockerjava.zerodep.ZerodepDockerHttpClient import org.scalatest.concurrent.Eventually import org.scalatest.time.SpanSugar._ @@ -121,7 +122,12 @@ abstract class DockerJDBCIntegrationSuite override def beforeAll(): Unit = runIfTestsEnabled(s"Prepare for ${this.getClass.getName}") { super.beforeAll() try { - docker = DockerClientBuilder.getInstance().build() + val config = DefaultDockerClientConfig.createDefaultConfigBuilder.build + val httpClient = new ZerodepDockerHttpClient.Builder() + .dockerHost(config.getDockerHost) + .sslConfig(config.getSSLConfig) + .build() + docker = DockerClientImpl.getInstance(config, httpClient) // Check that Docker is actually up try { docker.pingCmd().exec() diff --git a/pom.xml b/pom.xml index cb5b514909ce..f0da42604879 100644 --- a/pom.xml +++ b/pom.xml @@ -1250,8 +1250,18 @@ com.github.docker-java docker-java-transport-netty + + com.github.docker-java + docker-java-transport-jersey + + + com.github.docker-java + docker-java-transport-zerodep + 3.3.4 + test + com.mysql mysql-connector-j From 88bcd5c0d06b90ffd78c00349b5424d1891da2c7 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Dec 2023 23:10:51 +0800 Subject: [PATCH 5/6] Update connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala --- .../org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index d7b52106f3ba..3c32df424c0c 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -138,7 +138,7 @@ abstract class DockerJDBCIntegrationSuite } try { // Ensure that the Docker image is installed: - docker.inspectImageCmd("mariadb:10.5.12").exec() + docker.inspectImageCmd(db.imageName).exec() } catch { case e: NotFoundException => log.warn(s"Docker image ${db.imageName} not found; pulling image from registry") From 5a70542f9f5d5f19bf6828156ea03c2767f3751b Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Dec 2023 23:14:52 +0800 Subject: [PATCH 6/6] try-finally --- .../sql/jdbc/DockerJDBCIntegrationSuite.scala | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index 3c32df424c0c..fde228939dd4 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -249,19 +249,15 @@ abstract class DockerJDBCIntegrationSuite } private def logContainerOutput(): Unit = { - try { - logInfo("\n\n===== CONTAINER LOGS FOR container Id: " + container + " =====") - - docker.logContainerCmd(container.getId) - .withStdOut(true) - .withStdErr(true) - .withFollowStream(true) - .withSince(0) - .exec(new ResultCallbackTemplate[ResultCallback[Frame], Frame] { - override def onNext(f: Frame): Unit = logInfo(f.toString) - }) - logInfo("\n\n===== END OF CONTAINER LOGS FOR container Id: " + container + " =====") - } finally { - } + logInfo("\n\n===== CONTAINER LOGS FOR container Id: " + container + " =====") + docker.logContainerCmd(container.getId) + .withStdOut(true) + .withStdErr(true) + .withFollowStream(true) + .withSince(0).exec( + new ResultCallbackTemplate[ResultCallback[Frame], Frame] { + override def onNext(f: Frame): Unit = logInfo(f.toString) + }) + logInfo("\n\n===== END OF CONTAINER LOGS FOR container Id: " + container + " =====") } }