diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 86757dd..13672cb 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -24,7 +24,7 @@ jobs: strategy: matrix: java-version: [ 8, 11 ] - runs-on: [ubuntu-latest, macos-latest, windows-latest ] + runs-on: [ubuntu-latest] runs-on: ${{ matrix.runs-on }} steps: - uses: actions/checkout@v2 @@ -34,5 +34,7 @@ jobs: java-version: ${{ matrix.java-version }} distribution: 'adopt' cache: maven + - name: Set up Docker Buildx + uses: docker/setup-buildx-action@v1 - name: Build with Maven run: ./mvnw verify diff --git a/flink-connector-clickhouse-e2e-test/pom.xml b/flink-connector-clickhouse-e2e-test/pom.xml index 860d69a..d2285ed 100644 --- a/flink-connector-clickhouse-e2e-test/pom.xml +++ b/flink-connector-clickhouse-e2e-test/pom.xml @@ -35,6 +35,137 @@ limitations under the License. - + + org.apache.flink + flink-connector-test-utils + ${flink.version} + + + org.testcontainers + clickhouse + ${testcontainer.version} + + + com.clickhouse + clickhouse-jdbc + ${clickhouse-jdbc.version} + test + + + org.apache.flink + flink-connector-clickhouse + ${project.version} + test + + + org.apache.httpcomponents.client5 + httpclient5 + ${httpclient5.version} + test + + + org.apache.httpcomponents.core5 + httpcore5 + ${httpcore5.version} + test + + + + org.apache.httpcomponents.core5 + httpcore5-h2 + ${httpcore5.version} + test + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + default-test + none + + + integration-tests + none + + + end-to-end-tests + integration-test + + test + + + + **/*.* + + 1 + + ${project.basedir} + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-jars + process-resources + + copy + + + + + + + org.apache.flink + flink-connector-clickhouse + ${project.version} + jar + true + ${project.build.directory} + + + org.apache.httpcomponents.client5 + httpclient5 + 5.2.1 + jar + true + ${project.build.directory} + + + org.apache.httpcomponents.core5 + httpcore5 + 5.2 + jar + true + ${project.build.directory} + + + org.apache.httpcomponents.core5 + httpcore5-h2 + 5.2 + jar + true + ${project.build.directory} + + + com.clickhouse + clickhouse-jdbc + ${clickhouse-jdbc.version} + jar + true + ${project.build.directory} + + + + + + diff --git a/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/ClickhouseE2ECase.java b/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/ClickhouseE2ECase.java new file mode 100644 index 0000000..0f5072f --- /dev/null +++ b/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/ClickhouseE2ECase.java @@ -0,0 +1,97 @@ +package org.apache.flink.connector.clickhouse; + +import org.junit.After; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** End-to-end test for Clickhouse. */ +public class ClickhouseE2ECase extends FlinkContainerTestEnviroment { + + private static final Logger logger = LoggerFactory.getLogger(ClickhouseE2ECase.class); + + ClickhouseProxy proxy; + + @Test + public void testSink() throws Exception { + String jdbcUrl = String.format("jdbc:clickhouse://%s:%s/default", "clickhouse", "8123"); + + proxy = + new ClickhouseProxy( + CLICKHOUSE_CONTAINER.getJdbcUrl(), + CLICKHOUSE_CONTAINER.getUsername(), + CLICKHOUSE_CONTAINER.getPassword()); + proxy.execute( + "create table test (id Int32, name String, float32_column Float32, date_column Date,datetime_column DateTime, array_column Array(Int32)) engine = Memory"); + proxy.execute( + "create table test_insert (id Int32, name String, float32_column Float32, date_column Date,datetime_column DateTime, array_column Array(Int32)) engine = Memory; "); + proxy.execute( + "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (1, 'Name1', 1.1, '2022-01-01', '2022-01-01 00:00:00', [1, 2, 3]);"); + proxy.execute( + "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (2, 'Name2', 2.2, '2022-01-02', '2022-01-02 01:00:00', [4, 5, 6]);"); + proxy.execute( + "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (3, 'Name3', 3.3, '2022-01-03', '2022-01-03 02:00:00', [7, 8, 9]);"); + proxy.execute( + "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (4, 'Name4', 4.4, '2022-01-04', '2022-01-04 03:00:00', [10, 11, 12]);"); + proxy.execute( + "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (5, 'Name5', 5.5, '2022-01-05', '2022-01-05 04:00:00', [13, 14, 15]);"); + // proxy.execute("insert into test values (2, 'kiki');"); + List sqlLines = new ArrayList<>(); + sqlLines.add( + "create table clickhouse_test (id int, name varchar,float32_column FLOAT,\n" + + " datetime_column TIMESTAMP(3),\n" + + " array_column ARRAY) with ('connector' = 'clickhouse',\n" + + " 'url' = '" + + jdbcUrl + + "',\n" + + " 'table-name' = 'test',\n" + + " 'username'='test_username',\n" + + " 'password'='test_password'\n" + + ");"); + sqlLines.add( + "create table test (id int, name varchar,float32_column FLOAT,\n" + + " datetime_column TIMESTAMP(3),\n" + + " array_column ARRAY) with ('connector' = 'clickhouse',\n" + + " 'url' = '" + + jdbcUrl + + "',\n" + + " 'table-name' = 'test_insert',\n" + + " 'username'='test_username',\n" + + " 'password'='test_password'\n" + + ");"); + sqlLines.add("insert into test select * from clickhouse_test;"); + + submitSQLJob( + sqlLines, + SQL_CONNECTOR_CLICKHOUSE_JAR, + CLICKHOUSE_JDBC_JAR, + HTTPCORE_JAR, + HTTPCLIENT_JAR, + HTTPCLIENT_H2_JAR); + waitUntilJobRunning(Duration.of(1, ChronoUnit.MINUTES)); + List expectedResult = + Arrays.asList( + "1,Name1,1.1,2022-01-01 00:00:00,[1,2,3]", + "2,Name2,2.2,2022-01-02 01:00:00,[4,5,6]", + "3,Name3,3.3,2022-01-03 02:00:00,[7,8,9]", + "4,Name4,4.4,2022-01-04 03:00:00,[10,11,12]", + "5,Name5,5.5,2022-01-05 04:00:00,[13,14,15]"); + proxy.checkResultWithTimeout( + expectedResult, + "test_insert", + Arrays.asList("id", "name", "float32_column", "datetime_column", "array_column"), + 60000); + } + + @After + public void tearDown() throws SQLException { + CLICKHOUSE_CONTAINER.stop(); + } +} diff --git a/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/ClickhouseProxy.java b/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/ClickhouseProxy.java new file mode 100644 index 0000000..6bb70ae --- /dev/null +++ b/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/ClickhouseProxy.java @@ -0,0 +1,116 @@ +package org.apache.flink.connector.clickhouse; + +import com.clickhouse.jdbc.ClickHouseConnection; +import com.clickhouse.jdbc.ClickHouseDataSource; +import com.clickhouse.jdbc.ClickHouseDriver; +import com.clickhouse.jdbc.ClickHouseStatement; +import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Array; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; + +/** A proxy for Clickhouse to execute SQLs and check results. */ +public class ClickhouseProxy { + private String jdbcUrl; + private String username; + private String password; + private static final Logger logger = LoggerFactory.getLogger(ClickhouseProxy.class); + ClickHouseDriver driver; + ClickHouseStatement statement; + ClickHouseConnection connection; + + ClickhouseProxy(String jdbcUrl, String username, String password) { + this.jdbcUrl = jdbcUrl; + this.username = username; + this.password = password; + this.driver = new ClickHouseDriver(); + } + + public void connect() { + try { + if (connection == null) { + Properties properties = new Properties(); + properties.put("username", username); + properties.put("password", password); + ClickHouseDataSource clickHouseDataSource = + new ClickHouseDataSource(jdbcUrl, properties); + connection = clickHouseDataSource.getConnection(username, password); + statement = connection.createStatement(); + } + } catch (Exception e) { + logger.error("Failed to connect to clickhouse", e); + } + } + + public void execute(String sql) throws SQLException { + connect(); + statement.execute(sql); + } + + private void checkResult(List expectedResult, String table, List fields) + throws Exception { + connect(); + List results = new ArrayList<>(); + ResultSet resultSet = statement.executeQuery("select * from " + table); + while (resultSet.next()) { + List result = new ArrayList<>(); + ResultSetMetaData metaData = resultSet.getMetaData(); + int columnCount = metaData.getColumnCount(); + for (int i = 1; i <= columnCount; i++) { + String columnName = metaData.getColumnName(i); + if (!fields.contains(columnName)) { + continue; + } + String columnType = metaData.getColumnTypeName(i); + switch (columnType) { + case "Array": + Array array = resultSet.getArray(i); + result.add(array.toString()); + break; + case "Timestamp": + Timestamp timestamp = resultSet.getTimestamp(i); + result.add(timestamp.toString()); + break; + default: + String value = resultSet.getString(i); + result.add(value); + break; + } + } + + results.add(result.stream().collect(Collectors.joining(","))); + } + Collections.sort(results); + Collections.sort(expectedResult); + Assert.assertArrayEquals(expectedResult.toArray(), results.toArray()); + } + + public void checkResultWithTimeout( + List expectedResult, String table, List fields, long timeout) + throws Exception { + long endTimeout = System.currentTimeMillis() + timeout; + boolean result = false; + while (System.currentTimeMillis() < endTimeout) { + try { + checkResult(expectedResult, table, fields); + result = true; + break; + } catch (AssertionError | SQLException throwable) { + Thread.sleep(1000L); + } + } + if (!result) { + checkResult(expectedResult, table, fields); + } + } +} diff --git a/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/FlinkContainerTestEnviroment.java b/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/FlinkContainerTestEnviroment.java new file mode 100644 index 0000000..13d09ee --- /dev/null +++ b/flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/FlinkContainerTestEnviroment.java @@ -0,0 +1,219 @@ +package org.apache.flink.connector.clickhouse; + +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.client.deployment.StandaloneClusterId; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.connector.testframe.container.TestcontainersSettings; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.test.resources.ResourceTestUtils; +import org.apache.flink.test.util.SQLJobSubmission; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.clickhouse.ClickHouseContainer; +import org.testcontainers.containers.Container; +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.DockerImageName; +import org.testcontainers.utility.MountableFile; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.Duration; +import java.util.ArrayList; +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.assertj.core.util.Preconditions.checkState; + +/** Test environment running job on Flink containers. */ +public class FlinkContainerTestEnviroment { + + private static final Logger logger = + LoggerFactory.getLogger(FlinkContainerTestEnviroment.class); + public static final Network NETWORK = Network.newNetwork(); + + static final ClickHouseContainer CLICKHOUSE_CONTAINER = + new ClickHouseContainer("clickhouse/clickhouse-server:latest") + .withNetwork(NETWORK) + .withNetworkAliases("clickhouse") + .withExposedPorts(8123, 9000) + .withUsername("test_username") + .withPassword("test_password") + .withLogConsumer(new Slf4jLogConsumer(logger)); + + private static final TestcontainersSettings TESTCONTAINERS_SETTINGS = + TestcontainersSettings.builder() + .logger(logger) + .network(NETWORK) + .dependsOn(CLICKHOUSE_CONTAINER) + .build(); + + public static final Path SQL_CONNECTOR_CLICKHOUSE_JAR = + ResourceTestUtils.getResource("flink-connector-clickhouse-1.0.0-SNAPSHOT.jar"); + public static final Path CLICKHOUSE_JDBC_JAR = + ResourceTestUtils.getResource("clickhouse-jdbc-0.6.1.jar"); + public static final Path HTTPCORE_JAR = ResourceTestUtils.getResource("httpcore5-5.2.jar"); + public static final Path HTTPCLIENT_JAR = + ResourceTestUtils.getResource("httpclient5-5.2.1.jar"); + public static final Path HTTPCLIENT_H2_JAR = + ResourceTestUtils.getResource("httpcore5-h2-5.2.jar"); + @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private GenericContainer jobManager; + private GenericContainer taskManager; + protected RestClusterClient restClusterClient; + + @Before + public void setUp() throws Exception { + CLICKHOUSE_CONTAINER.start(); + + String properties = + String.join( + "\n", + Arrays.asList( + "jobmanager.rpc.address: jobmanager", + "heartbeat.timeout: 60000", + "parallelism.default: 1")); + jobManager = + new GenericContainer<>(new DockerImageName("flink:1.19.0-scala_2.12")) + .withCommand("jobmanager") + .withNetwork(NETWORK) + .withExtraHost("host.docker.internal", "host-gateway") + .withNetworkAliases("jobmanager") + .withExposedPorts(8081, 6123) + .dependsOn(CLICKHOUSE_CONTAINER) + .withLabel("com.testcontainers.allow-filesystem-access", "true") + .withEnv("FLINK_PROPERTIES", properties) + .withLogConsumer(new Slf4jLogConsumer(logger)); + taskManager = + new GenericContainer<>(new DockerImageName("flink:1.19.0-scala_2.12")) + .withCommand("taskmanager") + .withExtraHost("host.docker.internal", "host-gateway") + .withNetwork(NETWORK) + .withNetworkAliases("taskmanager") + .withEnv("FLINK_PROPERTIES", properties) + .dependsOn(jobManager) + .withLabel("com.testcontainers.allow-filesystem-access", "true") + .withLogConsumer(new Slf4jLogConsumer(logger)); + Startables.deepStart(Stream.of(jobManager)).join(); + Startables.deepStart(Stream.of(taskManager)).join(); + Thread.sleep(5000); + logger.info("Containers are started."); + } + + /** + * Returns the {@link RestClusterClient} for the running cluster. + * + *

NOTE: The client is created lazily and should only be retrieved after the cluster + * is running. + */ + public RestClusterClient getRestClusterClient() { + if (restClusterClient != null) { + return restClusterClient; + } + checkState( + 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.PORT, jobManager.getMappedPort(8081)); + this.restClusterClient = + new RestClusterClient<>(clientConfiguration, StandaloneClusterId.getInstance()); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to create client for Flink container cluster", e); + } + return restClusterClient; + } + + /** + * Submits a SQL job to the running cluster. + * + *

NOTE: You should not use {@code '\t'}. + */ + public void submitSQLJob(List sqlLines, Path... jars) + throws IOException, InterruptedException { + logger.info("submitting flink sql task"); + + SQLJobSubmission job = + new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines).addJars(jars).build(); + final List commands = new ArrayList<>(); + Path script = temporaryFolder.newFile().toPath(); + Files.write(script, job.getSqlLines()); + jobManager.copyFileToContainer(MountableFile.forHostPath(script), "/tmp/script.sql"); + commands.add("cat /tmp/script.sql | "); + commands.add("bin/sql-client.sh"); + for (String jar : job.getJars()) { + commands.add("--jar"); + String containerPath = copyAndGetContainerPath(jobManager, jar); + commands.add(containerPath); + } + + Container.ExecResult execResult = + jobManager.execInContainer("bash", "-c", String.join(" ", commands)); + logger.info("execute result:" + execResult.getStdout()); + logger.error("execute error:" + execResult.getStderr()); + if (execResult.getExitCode() != 0) { + throw new AssertionError("Failed when submitting the SQL job."); + } + } + + /* + * Copy the file to the container and return the container path. + */ + private String copyAndGetContainerPath(GenericContainer container, String filePath) { + Path path = Paths.get(filePath); + String containerPath = "/tmp/" + path.getFileName(); + container.copyFileToContainer(MountableFile.forHostPath(path), containerPath); + return containerPath; + } + + private static List readSqlFile(final String resourceName) throws Exception { + return Files.readAllLines( + Paths.get(ClickhouseE2ECase.class.getResource("/" + resourceName).toURI())); + } + + public void waitUntilJobRunning(Duration timeout) { + RestClusterClient clusterClient = getRestClusterClient(); + Deadline deadline = Deadline.fromNow(timeout); + while (deadline.hasTimeLeft()) { + Collection jobStatusMessages; + try { + jobStatusMessages = clusterClient.listJobs().get(10, TimeUnit.SECONDS); + } catch (Exception e) { + logger.warn("Error when fetching job status.", e); + continue; + } + if (jobStatusMessages != null && !jobStatusMessages.isEmpty()) { + JobStatusMessage message = jobStatusMessages.iterator().next(); + JobStatus jobStatus = message.getJobState(); + if (jobStatus.isTerminalState() && message.getJobState() == JobStatus.FAILED) { + throw new ValidationException( + String.format( + "Job has been terminated! JobName: %s, JobID: %s, Status: %s", + message.getJobName(), + message.getJobId(), + message.getJobState())); + } else if (jobStatus == JobStatus.RUNNING) { + return; + } + } + } + } +} diff --git a/pom.xml b/pom.xml index 7881d18..c1aeee3 100644 --- a/pom.xml +++ b/pom.xml @@ -68,6 +68,9 @@ limitations under the License. 32.1.3-jre 2.12.4 3.13.0 + 1.19.8 + 5.2.1 + 5.2 org.apache.flink.shaded.clickhouse flink-connector-clickhouse-parent 3.8.1