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

[FLINK-34639] Support debezium deserializer in OceanBase source connector #3124

Merged
merged 4 commits into from
Apr 25, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
265 changes: 165 additions & 100 deletions docs/content.zh/docs/connectors/legacy-flink-cdc-sources/oceanbase-cdc.md

Large diffs are not rendered by default.

207 changes: 131 additions & 76 deletions docs/content/docs/connectors/legacy-flink-cdc-sources/oceanbase-cdc.md

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,9 @@ limitations under the License.
<version>${revision}</version>
</parent>
<properties>
<oblogclient.version>1.1.2</oblogclient.version>
<!-- Because of oceanbase docker image can not expose port quickly, so we need to specify testcontainers version to 1.15.3 -->
<jdbc.version>1.15.3</jdbc.version>
<testcontainers.version>1.15.3</testcontainers.version>
whhe marked this conversation as resolved.
Show resolved Hide resolved
</properties>
<modelVersion>4.0.0</modelVersion>

Expand All @@ -34,6 +35,13 @@ limitations under the License.
<packaging>jar</packaging>

<dependencies>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-base</artifactId>
<version>${project.version}</version>
</dependency>

<!-- Debezium dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
Expand All @@ -47,12 +55,6 @@ limitations under the License.
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-common</artifactId>
<version>${project.version}</version>
</dependency>

<!-- OceanBase Log Client -->
<dependency>
<groupId>com.oceanbase</groupId>
Expand All @@ -64,7 +66,13 @@ limitations under the License.
<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
<version>5.1.47</version>
<version>8.0.27</version>
<exclusions>
<exclusion>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- test dependencies on Flink -->
Expand Down Expand Up @@ -151,7 +159,7 @@ limitations under the License.
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>jdbc</artifactId>
<version>${jdbc.version}</version>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,13 @@
package org.apache.flink.cdc.connectors.oceanbase;

import org.apache.flink.cdc.common.annotation.PublicEvolving;
import org.apache.flink.cdc.connectors.base.options.StartupOptions;
import org.apache.flink.cdc.connectors.oceanbase.source.OceanBaseRichSourceFunction;
import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseDeserializationSchema;
import org.apache.flink.cdc.connectors.oceanbase.table.StartupMode;
import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema;
import org.apache.flink.streaming.api.functions.source.SourceFunction;

import com.oceanbase.clogproxy.client.config.ClientConf;
import com.oceanbase.clogproxy.client.config.ObReaderConfig;
import com.oceanbase.clogproxy.client.util.ClientIdGenerator;
import org.apache.commons.lang3.StringUtils;

import java.time.Duration;
Expand All @@ -47,7 +46,7 @@ public static <T> Builder<T> builder() {
public static class Builder<T> {

// common config
private StartupMode startupMode;
private StartupOptions startupOptions;
private String username;
private String password;
private String tenantName;
Expand All @@ -73,11 +72,12 @@ public static class Builder<T> {
private String configUrl;
private String workingMode;
private Properties obcdcProperties;
private Properties debeziumProperties;

private OceanBaseDeserializationSchema<T> deserializer;
private DebeziumDeserializationSchema<T> deserializer;

public Builder<T> startupMode(StartupMode startupMode) {
this.startupMode = startupMode;
public Builder<T> startupOptions(StartupOptions startupOptions) {
this.startupOptions = startupOptions;
return this;
}

Expand Down Expand Up @@ -151,7 +151,7 @@ public Builder<T> logProxyHost(String logProxyHost) {
return this;
}

public Builder<T> logProxyPort(int logProxyPort) {
public Builder<T> logProxyPort(Integer logProxyPort) {
whhe marked this conversation as resolved.
Show resolved Hide resolved
this.logProxyPort = logProxyPort;
return this;
}
Expand Down Expand Up @@ -186,23 +186,44 @@ public Builder<T> obcdcProperties(Properties obcdcProperties) {
return this;
}

public Builder<T> deserializer(OceanBaseDeserializationSchema<T> deserializer) {
public Builder<T> debeziumProperties(Properties debeziumProperties) {
this.debeziumProperties = debeziumProperties;
return this;
}

public Builder<T> deserializer(DebeziumDeserializationSchema<T> deserializer) {
this.deserializer = deserializer;
return this;
}

public SourceFunction<T> build() {
switch (startupMode) {
case INITIAL:
checkNotNull(hostname, "hostname shouldn't be null on startup mode 'initial'");
checkNotNull(port, "port shouldn't be null on startup mode 'initial'");
checkNotNull(
compatibleMode,
"compatibleMode shouldn't be null on startup mode 'initial'");
checkNotNull(
jdbcDriver, "jdbcDriver shouldn't be null on startup mode 'initial'");
startupTimestamp = 0L;
checkNotNull(username, "username shouldn't be null");
checkNotNull(password, "password shouldn't be null");
checkNotNull(hostname, "hostname shouldn't be null");
checkNotNull(port, "port shouldn't be null");

if (startupOptions == null) {
startupOptions = StartupOptions.initial();
}
if (compatibleMode == null) {
compatibleMode = "mysql";
}
if (jdbcDriver == null) {
jdbcDriver = "com.mysql.cj.jdbc.Driver";
}

if (connectTimeout == null) {
connectTimeout = Duration.ofSeconds(30);
}

if (serverTimeZone == null) {
serverTimeZone = "+00:00";
whhe marked this conversation as resolved.
Show resolved Hide resolved
}

switch (startupOptions.startupMode) {
case SNAPSHOT:
break;
case INITIAL:
case LATEST_OFFSET:
startupTimestamp = 0L;
break;
Expand All @@ -213,15 +234,9 @@ public SourceFunction<T> build() {
break;
default:
throw new UnsupportedOperationException(
startupMode + " mode is not supported.");
startupOptions.startupMode + " mode is not supported.");
}

if (!startupMode.equals(StartupMode.INITIAL)
&& (StringUtils.isNotEmpty(databaseName)
|| StringUtils.isNotEmpty(tableName))) {
throw new IllegalArgumentException(
"If startup mode is not 'INITIAL', 'database-name' and 'table-name' must not be configured");
}
if (StringUtils.isNotEmpty(databaseName) || StringUtils.isNotEmpty(tableName)) {
if (StringUtils.isEmpty(databaseName) || StringUtils.isEmpty(tableName)) {
throw new IllegalArgumentException(
Expand All @@ -233,57 +248,46 @@ public SourceFunction<T> build() {
"'database-name', 'table-name' or 'table-list' should be configured");
}

if (serverTimeZone == null) {
serverTimeZone = "+00:00";
}
ClientConf clientConf = null;
ObReaderConfig obReaderConfig = null;

if (connectTimeout == null) {
connectTimeout = Duration.ofSeconds(30);
}
if (!startupOptions.isSnapshotOnly()) {

if (logProxyClientId == null) {
logProxyClientId =
String.format(
"%s_%s_%s",
ClientIdGenerator.generate(),
Thread.currentThread().getId(),
checkNotNull(tenantName));
}
ClientConf clientConf =
ClientConf.builder()
.clientId(logProxyClientId)
.connectTimeoutMs((int) connectTimeout.toMillis())
.build();

ObReaderConfig obReaderConfig = new ObReaderConfig();
if (StringUtils.isNotEmpty(rsList)) {
obReaderConfig.setRsList(rsList);
}
if (StringUtils.isNotEmpty(configUrl)) {
obReaderConfig.setClusterUrl(configUrl);
}
if (StringUtils.isNotEmpty(workingMode)) {
obReaderConfig.setWorkingMode(workingMode);
}
obReaderConfig.setUsername(username);
obReaderConfig.setPassword(password);
obReaderConfig.setStartTimestamp(startupTimestamp);
obReaderConfig.setTimezone(serverTimeZone);

if (obcdcProperties != null && !obcdcProperties.isEmpty()) {
Map<String, String> extraConfigs = new HashMap<>();
obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString()));
obReaderConfig.setExtraConfigs(extraConfigs);
checkNotNull(logProxyHost);
checkNotNull(logProxyPort);
checkNotNull(tenantName);

obReaderConfig = new ObReaderConfig();
if (StringUtils.isNotEmpty(rsList)) {
obReaderConfig.setRsList(rsList);
}
if (StringUtils.isNotEmpty(configUrl)) {
obReaderConfig.setClusterUrl(configUrl);
}
if (StringUtils.isNotEmpty(workingMode)) {
obReaderConfig.setWorkingMode(workingMode);
}
obReaderConfig.setUsername(username);
obReaderConfig.setPassword(password);
obReaderConfig.setStartTimestamp(startupTimestamp);
obReaderConfig.setTimezone(serverTimeZone);

if (obcdcProperties != null && !obcdcProperties.isEmpty()) {
Map<String, String> extraConfigs = new HashMap<>();
obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString()));
obReaderConfig.setExtraConfigs(extraConfigs);
}
}

return new OceanBaseRichSourceFunction<>(
StartupMode.INITIAL.equals(startupMode),
startupOptions,
username,
password,
tenantName,
databaseName,
tableName,
tableList,
serverTimeZone,
connectTimeout,
hostname,
port,
Expand All @@ -292,8 +296,9 @@ public SourceFunction<T> build() {
jdbcProperties,
logProxyHost,
logProxyPort,
clientConf,
logProxyClientId,
obReaderConfig,
debeziumProperties,
deserializer);
}
}
Expand Down
Loading
Loading