Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[INLONG-8358][Sort] Add kafka connector on flink 1.15 #8713

Merged
merged 19 commits into from
Dec 12, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,14 @@
<includes>
<include>sort-connector-hudi-v1.15-${project.version}.jar</include>
</includes>
<fileMode>0644</fileMode>
</fileSet>
<fileSet>
<directory>../inlong-sort/sort-flink/sort-flink-v1.15/sort-connectors/kafka/target</directory>
<outputDirectory>inlong-sort/connectors</outputDirectory>
<includes>
<include>sort-connector-kafka-v1.15-${project.version}.jar</include>
</includes>
<fileMode>0644</fileMode>
</fileSet>
</fileSets>
Expand Down
6 changes: 6 additions & 0 deletions inlong-sort/sort-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,12 @@
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.inlong</groupId>
<artifactId>sort-connector-kafka-v1.15</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<plugins>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,12 +46,14 @@
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>postgresql</artifactId>
<version>${testcontainers.version}</version>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>kafka</artifactId>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>mongodb</artifactId>
<version>${testcontainers.version}</version>
</dependency>
<dependency>
<groupId>org.mongodb</groupId>
Expand Down Expand Up @@ -223,6 +225,14 @@
<type>jar</type>
<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
</artifactItem>
<artifactItem>
<groupId>org.apache.inlong</groupId>
<artifactId>sort-connector-kafka-v1.15</artifactId>
<version>${project.version}</version>
<destFileName>sort-connector-kafka.jar</destFileName>
<type>jar</type>
<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
</artifactItem>
</artifactItems>
</configuration>
<executions>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,230 @@
/*
* 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.inlong.sort.tests;

import org.apache.inlong.sort.tests.utils.FlinkContainerTestEnv;
import org.apache.inlong.sort.tests.utils.JdbcProxy;
import org.apache.inlong.sort.tests.utils.MySqlContainer;
import org.apache.inlong.sort.tests.utils.PlaceholderResolver;
import org.apache.inlong.sort.tests.utils.StarRocksContainer;
import org.apache.inlong.sort.tests.utils.TestUtils;

import org.junit.AfterClass;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.Container.ExecResult;
import org.testcontainers.containers.KafkaContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.utility.DockerImageName;

import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.time.Duration;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;

import static org.apache.inlong.sort.tests.utils.StarRocksManager.INTER_CONTAINER_STAR_ROCKS_ALIAS;
import static org.apache.inlong.sort.tests.utils.StarRocksManager.STAR_ROCKS_LOG;
import static org.apache.inlong.sort.tests.utils.StarRocksManager.buildStarRocksImage;
import static org.apache.inlong.sort.tests.utils.StarRocksManager.getNewStarRocksImageName;
import static org.apache.inlong.sort.tests.utils.StarRocksManager.initializeStarRocksTable;

/**
* End-to-end tests for sort-connector-kafka uber jar.
*/
public class KafkaE2EITCase extends FlinkContainerTestEnv {

private static final Logger LOG = LoggerFactory.getLogger(KafkaE2EITCase.class);

public static final Logger MYSQL_LOG = LoggerFactory.getLogger(MySqlContainer.class);

public static final Logger KAFKA_LOG = LoggerFactory.getLogger(KafkaContainer.class);

private static final Path kafkaJar = TestUtils.getResource("sort-connector-kafka.jar");
private static final Path mysqlJar = TestUtils.getResource("sort-connector-mysql-cdc.jar");
private static final Path starrocksJar = TestUtils.getResource("sort-connector-starrocks.jar");
private static final Path mysqlJdbcJar = TestUtils.getResource("mysql-driver.jar");

private static final String sqlFile;

static {
try {
URI kafkaSqlFile =
Objects.requireNonNull(KafkaE2EITCase.class.getResource("/flinkSql/kafka_test.sql")).toURI();
sqlFile = Paths.get(kafkaSqlFile).toString();
buildStarRocksImage();
} catch (URISyntaxException e) {
throw new RuntimeException(e);
}
}

@ClassRule
public static final KafkaContainer KAFKA =
new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.1"))
.withNetwork(NETWORK)
.withNetworkAliases("kafka")
.withEmbeddedZookeeper()
.withLogConsumer(new Slf4jLogConsumer(KAFKA_LOG));

@ClassRule
public static StarRocksContainer STAR_ROCKS =
(StarRocksContainer) new StarRocksContainer(getNewStarRocksImageName())
.withExposedPorts(9030, 8030, 8040)
.withNetwork(NETWORK)
.withAccessToHost(true)
.withNetworkAliases(INTER_CONTAINER_STAR_ROCKS_ALIAS)
.withLogConsumer(new Slf4jLogConsumer(STAR_ROCKS_LOG));

@ClassRule
public static final MySqlContainer MYSQL_CONTAINER =
(MySqlContainer) new MySqlContainer(MySqlContainer.MySqlVersion.V8_0)
.withDatabaseName("test")
.withNetwork(NETWORK)
.withNetworkAliases("mysql")
.withLogConsumer(new Slf4jLogConsumer(MYSQL_LOG));

@Before
public void setup() {
waitUntilJobRunning(Duration.ofSeconds(30));
initializeMysqlTable();
initializeStarRocksTable(STAR_ROCKS);
}

private void initializeMysqlTable() {
try {
Class.forName(MYSQL_CONTAINER.getDriverClassName());
Connection conn = DriverManager
.getConnection(MYSQL_CONTAINER.getJdbcUrl(), MYSQL_CONTAINER.getUsername(),
MYSQL_CONTAINER.getPassword());
Statement stat = conn.createStatement();
stat.execute(
"CREATE TABLE test_input (\n"
+ " id SERIAL,\n"
+ " name VARCHAR(255) NOT NULL DEFAULT 'flink',\n"
+ " description VARCHAR(512),\n"
+ " PRIMARY KEY(id)\n"
+ ");");
stat.close();
conn.close();
} catch (Exception e) {
throw new RuntimeException(e);
}
}

@AfterClass
public static void teardown() {
if (KAFKA != null) {
KAFKA.stop();
}

if (MYSQL_CONTAINER != null) {
MYSQL_CONTAINER.stop();
}

if (STAR_ROCKS != null) {
STAR_ROCKS.stop();
}
}

private void initializeKafkaTable(String topic) {
String fileName = "kafka_test_kafka_init.txt";
int port = KafkaContainer.ZOOKEEPER_PORT;

Map<String, Object> properties = new HashMap<>();
properties.put("TOPIC", topic);
properties.put("ZOOKEEPER_PORT", port);

try {
String createKafkaStatement = getCreateStatement(fileName, properties);
ExecResult result = KAFKA.execInContainer("bash", "-c", createKafkaStatement);
LOG.info("Create kafka topic: {}, std: {}", createKafkaStatement, result.getStdout());
if (result.getExitCode() != 0) {
throw new RuntimeException("Init kafka topic failed. Exit code:" + result.getExitCode());
}
} catch (IOException | InterruptedException e) {
throw new RuntimeException(e);
}
}

private String getCreateStatement(String fileName, Map<String, Object> properties) {
URL url = Objects.requireNonNull(KafkaE2EITCase.class.getResource("/env/" + fileName));

try {
Path file = Paths.get(url.toURI());
return PlaceholderResolver.getDefaultResolver().resolveByMap(
new String(Files.readAllBytes(file), StandardCharsets.UTF_8),
properties);
} catch (IOException | URISyntaxException e) {
throw new RuntimeException(e);
}
}

/**
* Test flink sql mysql cdc to starrocks.
*
* @throws Exception The exception may throw when execute the case
*/
@Test
public void testKafkaWithSqlFile() throws Exception {
final String topic = "test-topic";
initializeKafkaTable(topic);

submitSQLJob(sqlFile, kafkaJar, starrocksJar, mysqlJar, mysqlJdbcJar);
waitUntilJobRunning(Duration.ofSeconds(10));

// generate input
try (Connection conn = DriverManager.getConnection(MYSQL_CONTAINER.getJdbcUrl(),
MYSQL_CONTAINER.getUsername(), MYSQL_CONTAINER.getPassword());
Statement stat = conn.createStatement()) {
stat.execute("INSERT INTO test_input VALUES (1,'jacket','water resistant white wind breaker');");
stat.execute("INSERT INTO test_input VALUES (2,'scooter','Big 2-wheel scooter ');");
} catch (SQLException e) {
LOG.error("Update table for CDC failed.", e);
throw e;
}

JdbcProxy proxy = new JdbcProxy(STAR_ROCKS.getJdbcUrl(), STAR_ROCKS.getUsername(),
STAR_ROCKS.getPassword(),
STAR_ROCKS.getDriverClassName());

List<String> expectResult = Arrays.asList(
"1,jacket,water resistant white wind breaker",
"2,scooter,Big 2-wheel scooter ");
proxy.checkResultWithTimeout(
expectResult,
"test_output1",
3,
60000L);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@
import java.sql.SQLException;
import java.sql.Statement;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;

import static org.apache.inlong.sort.tests.utils.StarRocksManager.INTER_CONTAINER_STAR_ROCKS_ALIAS;
Expand Down Expand Up @@ -155,12 +155,11 @@ public void testMysqlUpdateAndDelete() throws Exception {
throw e;
}

JdbcProxy proxy =
new JdbcProxy(STAR_ROCKS.getJdbcUrl(), STAR_ROCKS.getUsername(),
STAR_ROCKS.getPassword(),
STAR_ROCKS.getDriverClassName());
List<String> expectResult =
Arrays.asList("2,tom,Big 2-wheel scooter ");
JdbcProxy proxy = new JdbcProxy(STAR_ROCKS.getJdbcUrl(), STAR_ROCKS.getUsername(),
STAR_ROCKS.getPassword(),
STAR_ROCKS.getDriverClassName());

List<String> expectResult = Collections.singletonList("2,tom,Big 2-wheel scooter ");
proxy.checkResultWithTimeout(
expectResult,
"test_output1",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
kafka-topics --create --topic ${TOPIC} --replication-factor 1 --partitions 1 --zookeeper localhost:${ZOOKEEPER_PORT}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
CREATE TABLE test_input (
`id` INT primary key,
name STRING,
description STRING
) WITH (
'connector' = 'mysql-cdc-inlong',
'hostname' = 'mysql',
'port' = '3306',
'username' = 'root',
'password' = 'inlong',
'database-name' = 'test',
'table-name' = 'test_input',
'scan.incremental.snapshot.enabled' = 'false',
'jdbc.properties.useSSL' = 'false',
'jdbc.properties.allowPublicKeyRetrieval' = 'true'
);

CREATE TABLE kafka_load (
`id` INT NOT NULL primary key,
name STRING,
description STRING
) WITH (
'connector' = 'upsert-kafka-inlong',
'topic' = 'test-topic',
'properties.bootstrap.servers' = 'kafka:9092',
'key.format' = 'csv',
'value.format' = 'csv'
);

CREATE TABLE kafka_extract (
`id` INT NOT NULL,
name STRING,
description STRING
) WITH (
'connector' = 'kafka-inlong',
'topic' = 'test-topic',
'properties.bootstrap.servers' = 'kafka:9092',
'properties.group.id' = 'testGroup',
'scan.startup.mode' = 'earliest-offset',
'format' = 'csv'
);

CREATE TABLE test_output (
`id` INT primary key,
name STRING,
description STRING
) WITH (
'connector' = 'starrocks-inlong',
'jdbc-url' = 'jdbc:mysql://starrocks:9030',
'load-url'='starrocks:8030',
'database-name'='test',
'table-name' = 'test_output1',
'username' = 'inlong',
'password' = 'inlong',
'sink.properties.format' = 'json',
'sink.properties.strip_outer_array' = 'true',
'sink.buffer-flush.interval-ms' = '1000'
);

INSERT INTO kafka_load select * from test_input;
INSERT INTO test_output select * from kafka_extract;
Loading