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

[WIP][db2] The first draft of DB2-CDC #450

Closed
wants to merge 1 commit into from
Closed
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
137 changes: 137 additions & 0 deletions flink-connector-db2-cdc/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>flink-cdc-connectors</artifactId>
<groupId>com.ververica</groupId>
<version>2.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>flink-connector-db2-cdc</artifactId>
<name>flink-connector-db2-cdc</name>
<packaging>jar</packaging>

<dependencies>
<!-- Debezium dependencies -->
<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-connector-debezium</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
<artifactId>kafka-log4j-appender</artifactId>
<groupId>org.apache.kafka</groupId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-connector-db2</artifactId>
<version>${debezium.version}</version>
</dependency>

<!-- Test dependencies -->
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>db2</artifactId>
<version>1.15.1</version>
<scope>test</scope>
</dependency>

<!-- db2 driver dependency -->
<dependency>
<groupId>com.ibm.db2.jcc</groupId>
<artifactId>db2jcc</artifactId>
<version>db2jcc4</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>


<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-connector-test-util</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<scope>test</scope>
<type>test-jar</type>
<version>${flink.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-tests</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
<version>2.4.0</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,135 @@
/*
* 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 com.ververica.cdc.connectors.db2;

import com.ververica.cdc.connectors.db2.table.StartupOptions;
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
import com.ververica.cdc.debezium.DebeziumSourceFunction;
import com.ververica.cdc.debezium.Validator;
import io.debezium.connector.db2.Db2Connector;

import java.util.Properties;

import static org.apache.flink.util.Preconditions.checkNotNull;

/** Source for DB2 CDC connector. */
public class Db2Source {

public static <T> Builder<T> builder() {
return new Builder<>();
}

/**
* Builder for Db2Source.
*
* @param <T> Output type of the source
*/
public static class Builder<T> {
private String hostname;
private int port = 50000;
private String username;
private String password;
private String database;
// Should be in "schema.table" format
private String[] tableList;
private Properties dbzProperties;
private StartupOptions startupOptions = StartupOptions.initial();
private DebeziumDeserializationSchema<T> deserializer;

public DebeziumSourceFunction<T> build() {
Properties props = new Properties();
props.setProperty("connector.class", Db2Connector.class.getCanonicalName());
props.setProperty("database.hostname", checkNotNull(hostname));
props.setProperty("database.port", String.valueOf(port));
props.setProperty("database.user", checkNotNull(username));
props.setProperty("database.password", checkNotNull(password));
props.setProperty("database.dbname", checkNotNull(database));
props.setProperty("database.server.name", "db2_cdc_source"); // Hard-coded here
props.setProperty("database.history.skip.unparseable.ddl", String.valueOf(true));

if (tableList != null) {
props.setProperty("table.whitelist", String.join(",", tableList));
}
if (dbzProperties != null) {
dbzProperties.forEach(props::put);
}
switch (startupOptions.startupMode) {
case INITIAL:
props.setProperty("snapshot.mode", "initial");
break;

case LATEST_OFFSET:
props.setProperty("snapshot.mode", "schema_only");
break;

default:
throw new UnsupportedOperationException();
}

return new DebeziumSourceFunction<>(
deserializer, props, null, Validator.getDefaultValidator());
}

public Builder<T> hostname(String hostname) {
this.hostname = hostname;
return this;
}

public Builder<T> port(int port) {
this.port = port;
return this;
}

public Builder<T> username(String username) {
this.username = username;
return this;
}

public Builder<T> password(String password) {
this.password = password;
return this;
}

public Builder<T> database(String database) {
this.database = database;
return this;
}

public Builder<T> tableList(String... tableList) {
this.tableList = tableList;
return this;
}

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

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

/** Specifies the startup options. */
public Builder<T> startupOptions(StartupOptions startupOptions) {
this.startupOptions = startupOptions;
return this;
}
}
}
Loading