|
|
|
@ -35,12 +35,15 @@ import org.junit.Before;
|
|
|
|
|
import org.junit.ClassRule;
|
|
|
|
|
import org.junit.Rule;
|
|
|
|
|
import org.junit.rules.TemporaryFolder;
|
|
|
|
|
import org.junit.runner.RunWith;
|
|
|
|
|
import org.junit.runners.Parameterized;
|
|
|
|
|
import org.slf4j.Logger;
|
|
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
import org.testcontainers.containers.Container.ExecResult;
|
|
|
|
|
import org.testcontainers.containers.GenericContainer;
|
|
|
|
|
import org.testcontainers.containers.Network;
|
|
|
|
|
import org.testcontainers.containers.output.Slf4jLogConsumer;
|
|
|
|
|
import org.testcontainers.lifecycle.Startables;
|
|
|
|
|
import org.testcontainers.utility.MountableFile;
|
|
|
|
|
|
|
|
|
|
import javax.annotation.Nullable;
|
|
|
|
@ -55,19 +58,22 @@ import java.util.Arrays;
|
|
|
|
|
import java.util.Collection;
|
|
|
|
|
import java.util.List;
|
|
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
import java.util.stream.Stream;
|
|
|
|
|
|
|
|
|
|
import static org.apache.flink.util.Preconditions.checkState;
|
|
|
|
|
|
|
|
|
|
/** Test environment running job on Flink containers. */
|
|
|
|
|
@RunWith(Parameterized.class)
|
|
|
|
|
public abstract class FlinkContainerTestEnvironment extends TestLogger {
|
|
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(FlinkContainerTestEnvironment.class);
|
|
|
|
|
|
|
|
|
|
@Parameterized.Parameter public String flinkVersion;
|
|
|
|
|
|
|
|
|
|
// ------------------------------------------------------------------------------------------
|
|
|
|
|
// Flink Variables
|
|
|
|
|
// ------------------------------------------------------------------------------------------
|
|
|
|
|
private static final int JOB_MANAGER_REST_PORT = 8081;
|
|
|
|
|
private static final String FLINK_BIN = "bin";
|
|
|
|
|
private static final String FLINK_IMAGE_TAG = "flink:1.13.3-scala_2.11";
|
|
|
|
|
private static final String INTER_CONTAINER_JM_ALIAS = "jobmanager";
|
|
|
|
|
private static final String INTER_CONTAINER_TM_ALIAS = "taskmanager";
|
|
|
|
|
private static final String FLINK_PROPERTIES =
|
|
|
|
@ -91,6 +97,10 @@ public abstract class FlinkContainerTestEnvironment extends TestLogger {
|
|
|
|
|
|
|
|
|
|
@ClassRule public static final Network NETWORK = Network.newNetwork();
|
|
|
|
|
|
|
|
|
|
@Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder();
|
|
|
|
|
|
|
|
|
|
@Nullable private RestClusterClient<StandaloneClusterId> restClusterClient;
|
|
|
|
|
|
|
|
|
|
@ClassRule
|
|
|
|
|
public static final MySqlContainer MYSQL =
|
|
|
|
|
(MySqlContainer)
|
|
|
|
@ -104,36 +114,44 @@ public abstract class FlinkContainerTestEnvironment extends TestLogger {
|
|
|
|
|
.withNetworkAliases(INTER_CONTAINER_MYSQL_ALIAS)
|
|
|
|
|
.withLogConsumer(new Slf4jLogConsumer(LOG));
|
|
|
|
|
|
|
|
|
|
@Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder();
|
|
|
|
|
|
|
|
|
|
@Rule
|
|
|
|
|
public final GenericContainer<?> jobmanager =
|
|
|
|
|
new GenericContainer<>(FLINK_IMAGE_TAG)
|
|
|
|
|
.withCommand("jobmanager")
|
|
|
|
|
.withNetwork(NETWORK)
|
|
|
|
|
.withNetworkAliases(INTER_CONTAINER_JM_ALIAS)
|
|
|
|
|
.withExposedPorts(JOB_MANAGER_REST_PORT)
|
|
|
|
|
.withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES)
|
|
|
|
|
.withLogConsumer(new Slf4jLogConsumer(LOG));
|
|
|
|
|
|
|
|
|
|
@Rule
|
|
|
|
|
public final GenericContainer<?> taskmanager =
|
|
|
|
|
new GenericContainer<>(FLINK_IMAGE_TAG)
|
|
|
|
|
.withCommand("taskmanager")
|
|
|
|
|
.withNetwork(NETWORK)
|
|
|
|
|
.withNetworkAliases(INTER_CONTAINER_TM_ALIAS)
|
|
|
|
|
.withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES)
|
|
|
|
|
.dependsOn(jobmanager)
|
|
|
|
|
.withLogConsumer(new Slf4jLogConsumer(LOG));
|
|
|
|
|
|
|
|
|
|
@Nullable private RestClusterClient<StandaloneClusterId> restClusterClient;
|
|
|
|
|
|
|
|
|
|
protected final UniqueDatabase mysqlInventoryDatabase =
|
|
|
|
|
new UniqueDatabase(MYSQL, "mysql_inventory", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD);
|
|
|
|
|
protected Path jdbcJar;
|
|
|
|
|
|
|
|
|
|
private GenericContainer<?> jobManager;
|
|
|
|
|
private GenericContainer<?> taskManager;
|
|
|
|
|
|
|
|
|
|
@Parameterized.Parameters(name = "flinkVersion: {0}")
|
|
|
|
|
public static List<String> getFlinkVersion() {
|
|
|
|
|
return Arrays.asList("1.13.5", "1.14.3");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Before
|
|
|
|
|
public void before() {
|
|
|
|
|
mysqlInventoryDatabase.createAndInitialize();
|
|
|
|
|
jdbcJar = TestUtils.getResource(getJdbcConnectorResourceName());
|
|
|
|
|
|
|
|
|
|
LOG.info("Starting containers...");
|
|
|
|
|
jobManager =
|
|
|
|
|
new GenericContainer<>(getFlinkDockerImageTag())
|
|
|
|
|
.withCommand("jobmanager")
|
|
|
|
|
.withNetwork(NETWORK)
|
|
|
|
|
.withNetworkAliases(INTER_CONTAINER_JM_ALIAS)
|
|
|
|
|
.withExposedPorts(JOB_MANAGER_REST_PORT)
|
|
|
|
|
.withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES)
|
|
|
|
|
.withLogConsumer(new Slf4jLogConsumer(LOG));
|
|
|
|
|
taskManager =
|
|
|
|
|
new GenericContainer<>(getFlinkDockerImageTag())
|
|
|
|
|
.withCommand("taskmanager")
|
|
|
|
|
.withNetwork(NETWORK)
|
|
|
|
|
.withNetworkAliases(INTER_CONTAINER_TM_ALIAS)
|
|
|
|
|
.withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES)
|
|
|
|
|
.dependsOn(jobManager)
|
|
|
|
|
.withLogConsumer(new Slf4jLogConsumer(LOG));
|
|
|
|
|
|
|
|
|
|
Startables.deepStart(Stream.of(jobManager)).join();
|
|
|
|
|
Startables.deepStart(Stream.of(taskManager)).join();
|
|
|
|
|
LOG.info("Containers are started.");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@After
|
|
|
|
@ -141,6 +159,12 @@ public abstract class FlinkContainerTestEnvironment extends TestLogger {
|
|
|
|
|
if (restClusterClient != null) {
|
|
|
|
|
restClusterClient.close();
|
|
|
|
|
}
|
|
|
|
|
if (jobManager != null) {
|
|
|
|
|
jobManager.stop();
|
|
|
|
|
}
|
|
|
|
|
if (taskManager != null) {
|
|
|
|
|
taskManager.stop();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
@ -155,17 +179,17 @@ public abstract class FlinkContainerTestEnvironment extends TestLogger {
|
|
|
|
|
final List<String> commands = new ArrayList<>();
|
|
|
|
|
Path script = temporaryFolder.newFile().toPath();
|
|
|
|
|
Files.write(script, job.getSqlLines());
|
|
|
|
|
jobmanager.copyFileToContainer(MountableFile.forHostPath(script), "/tmp/script.sql");
|
|
|
|
|
jobManager.copyFileToContainer(MountableFile.forHostPath(script), "/tmp/script.sql");
|
|
|
|
|
commands.add("cat /tmp/script.sql | ");
|
|
|
|
|
commands.add(FLINK_BIN + "/sql-client.sh");
|
|
|
|
|
for (String jar : job.getJars()) {
|
|
|
|
|
commands.add("--jar");
|
|
|
|
|
String containerPath = copyAndGetContainerPath(jobmanager, jar);
|
|
|
|
|
String containerPath = copyAndGetContainerPath(jobManager, jar);
|
|
|
|
|
commands.add(containerPath);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
ExecResult execResult =
|
|
|
|
|
jobmanager.execInContainer("bash", "-c", String.join(" ", commands));
|
|
|
|
|
jobManager.execInContainer("bash", "-c", String.join(" ", commands));
|
|
|
|
|
LOG.info(execResult.getStdout());
|
|
|
|
|
LOG.error(execResult.getStderr());
|
|
|
|
|
if (execResult.getExitCode() != 0) {
|
|
|
|
@ -183,13 +207,13 @@ public abstract class FlinkContainerTestEnvironment extends TestLogger {
|
|
|
|
|
return restClusterClient;
|
|
|
|
|
}
|
|
|
|
|
checkState(
|
|
|
|
|
jobmanager.isRunning(),
|
|
|
|
|
jobManager.isRunning(),
|
|
|
|
|
"Cluster client should only be retrieved for a running cluster");
|
|
|
|
|
try {
|
|
|
|
|
final Configuration clientConfiguration = new Configuration();
|
|
|
|
|
clientConfiguration.set(RestOptions.ADDRESS, jobmanager.getHost());
|
|
|
|
|
clientConfiguration.set(RestOptions.ADDRESS, jobManager.getHost());
|
|
|
|
|
clientConfiguration.set(
|
|
|
|
|
RestOptions.PORT, jobmanager.getMappedPort(JOB_MANAGER_REST_PORT));
|
|
|
|
|
RestOptions.PORT, jobManager.getMappedPort(JOB_MANAGER_REST_PORT));
|
|
|
|
|
this.restClusterClient =
|
|
|
|
|
new RestClusterClient<>(clientConfiguration, StandaloneClusterId.getInstance());
|
|
|
|
|
} catch (Exception e) {
|
|
|
|
@ -233,4 +257,12 @@ public abstract class FlinkContainerTestEnvironment extends TestLogger {
|
|
|
|
|
container.copyFileToContainer(MountableFile.forHostPath(path), containerPath);
|
|
|
|
|
return containerPath;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private String getFlinkDockerImageTag() {
|
|
|
|
|
return String.format("flink:%s-scala_2.11", flinkVersion);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
protected String getJdbcConnectorResourceName() {
|
|
|
|
|
return String.format("jdbc-connector_%s.jar", flinkVersion);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|