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

[FEATURE][TOOLS-CDC]:add flink Command line support #107

Merged
merged 3 commits into from
Dec 11, 2024
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
5 changes: 5 additions & 0 deletions .github/workflows/push_pr.yml
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,11 @@ jobs:
with:
module: flink-connector-oceanbase-directload

flink-connector-oceanbase-tools-cdc:
uses: ./.github/workflows/test.yml
with:
module: flink-connector-oceanbase-tools-cdc

flink-connector-oceanbase-e2e-tests:
strategy:
matrix:
Expand Down
123 changes: 123 additions & 0 deletions flink-connector-oceanbase-tools-cdc/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Copyright 2024 OceanBase.

Licensed 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">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>com.oceanbase</groupId>
<artifactId>flink-connector-oceanbase-parent</artifactId>
<version>${revision}</version>
</parent>

<artifactId>flink-connector-oceanbase-tools-cdc</artifactId>
<packaging>jar</packaging>

<dependencies>
<dependency>
<groupId>com.oceanbase</groupId>
<artifactId>flink-connector-oceanbase</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>com.oceanbase</groupId>
<artifactId>flink-connector-oceanbase-base</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-base</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>mysql</artifactId>
<version>${testcontainer.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-mysql-cdc</artifactId>
<version>${flink.sql.cdc.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-guava</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-oracle-cdc</artifactId>
<version>${flink.sql.cdc.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-guava</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-postgres-cdc</artifactId>
<version>${flink.sql.cdc.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-guava</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-sqlserver-cdc</artifactId>
<version>${flink.sql.cdc.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-guava</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-db2-cdc</artifactId>
<version>${flink.sql.cdc.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-guava</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.oracle.ojdbc</groupId>
<artifactId>ojdbc8</artifactId>
<version>${ojdbc.version}</version>
<scope>provided</scope>
</dependency>
</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* Copyright 2024 OceanBase.
*
* Licensed 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.oceanbase.connector.flink.connection;

import com.oceanbase.connector.flink.OceanBaseConnectorOptions;
import com.oceanbase.connector.flink.tools.catalog.TableSchema;
import com.oceanbase.connector.flink.utils.OceanBaseToolsJdbcUtils;

public class OceanBaseToolsConnectProvider extends OceanBaseConnectionProvider {

public OceanBaseToolsConnectProvider(OceanBaseConnectorOptions options) {
super(options);
}

public boolean databaseExists(String database) {
return OceanBaseToolsJdbcUtils.databaseExists(database, this::getConnection);
}

public void createDatabase(String database) {
OceanBaseToolsJdbcUtils.createDatabase(database, this::getConnection);
}

public boolean tableExists(String database, String table) {
return OceanBaseToolsJdbcUtils.tableExists(database, table, this::getConnection);
}

public void createTable(TableSchema schema) {
OceanBaseToolsJdbcUtils.createTable(schema, this::getConnection);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,174 @@
/*
* Copyright 2024 OceanBase.
*
* Licensed 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.oceanbase.connector.flink.table;

import org.apache.flink.table.data.ArrayData;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.MapData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.TimestampData;
import org.apache.flink.table.types.logical.BooleanType;
import org.apache.flink.table.types.logical.IntType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.types.logical.VarCharType;

import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.sql.Date;
import java.sql.Time;
import java.time.LocalDate;
import java.time.LocalTime;
import java.time.ZoneId;
import java.util.stream.Collectors;
import java.util.stream.IntStream;

public class OceanBaseJsonSerializationSchema extends AbstractRecordSerializationSchema<String> {

private static final long serialVersionUID = 1L;
private static final Logger LOG =
LoggerFactory.getLogger(OceanBaseJsonSerializationSchema.class);

private final TableInfo tableInfo;
private static final ObjectMapper objectMapper = new ObjectMapper();

public OceanBaseJsonSerializationSchema(TableInfo tableInfo) {
this.tableInfo = tableInfo;
}

@Override
public Record serialize(String rowDataStr) {
try {
JsonNode rowDataNode = objectMapper.readTree(rowDataStr);
DataChangeRecord.Type type;
String op = rowDataNode.path("op").asText();
if ("r".equals(op) || "c".equals(op)) {
type = DataChangeRecord.Type.UPSERT;
} else if ("d".equals(op)) {
type = DataChangeRecord.Type.DELETE;
} else {
throw new IllegalArgumentException("Unknown operation type: " + op);
}
int size = tableInfo.getFieldNames().size();
Object[] values = new Object[size];
for (int i = 0; i < size; i++) {
String fieldName = tableInfo.getFieldNames().get(i);
JsonNode fieldNode = rowDataNode.path("after").path(fieldName);
values[i] = objectMapper.convertValue(fieldNode, new TypeReference<Object>() {});
}

return new DataChangeRecord(tableInfo, type, values);
} catch (IOException e) {
String errorMessage = "Failed to parse rowData JSON: " + rowDataStr;
LOG.error(errorMessage, e);
throw new RuntimeException(errorMessage);
}
}

@Override
protected SerializationRuntimeConverter createNotNullConverter(LogicalType type) {
switch (type.getTypeRoot()) {
case BOOLEAN:
case TINYINT:
case SMALLINT:
case INTEGER:
case INTERVAL_YEAR_MONTH:
case BIGINT:
case INTERVAL_DAY_TIME:
case FLOAT:
case DOUBLE:
case BINARY:
case VARBINARY:
return data -> data;
case CHAR:
case VARCHAR:
return Object::toString;
case DATE:
return data -> Date.valueOf(LocalDate.ofEpochDay((int) data));
case TIME_WITHOUT_TIME_ZONE:
return data -> Time.valueOf(LocalTime.ofNanoOfDay((int) data * 1_000_000L));
case TIMESTAMP_WITHOUT_TIME_ZONE:
return data -> ((TimestampData) data).toTimestamp();
case TIMESTAMP_WITH_TIME_ZONE:
return data -> ((TimestampData) data).toInstant().toString();
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return data ->
((TimestampData) data)
.toInstant()
.atZone(ZoneId.systemDefault())
.toString();
case DECIMAL:
return data -> ((DecimalData) data).toBigDecimal();
case ARRAY:
return data -> {
ArrayData arrayData = (ArrayData) data;
return IntStream.range(0, arrayData.size())
.mapToObj(i -> arrayData.getString(i).toString())
.collect(Collectors.joining(","));
};
case MAP:
return data -> {
MapData mapData = (MapData) data;
ArrayData keyArray = mapData.keyArray();
ArrayData valueArray = mapData.valueArray();
return "{"
+ IntStream.range(0, keyArray.size())
.mapToObj(
i ->
"\""
+ keyArray.getString(i).toString()
+ "\":\""
+ valueArray.getString(i).toString()
+ "\"")
.collect(Collectors.joining(","))
+ "}";
};
case ROW:
return data -> {
RowData rowData = (RowData) data;
RowType rowType = (RowType) type;
StringBuilder sb = new StringBuilder();
sb.append("{");
for (int i = 0; i < rowData.getArity(); i++) {
if (i > 0) {
sb.append(",");
}
String fieldName = rowType.getFieldNames().get(i);
LogicalType fieldType = rowType.getFields().get(i).getType();
sb.append("\"").append(fieldName).append("\":");
if (fieldType instanceof VarCharType) {
sb.append("\"").append(rowData.getString(i).toString()).append("\"");
} else if (fieldType instanceof IntType) {
sb.append(rowData.getInt(i));
} else if (fieldType instanceof BooleanType) {
sb.append(rowData.getBoolean(i));
}
// Add more types as needed
}
sb.append("}");
return sb.toString();
};
default:
throw new UnsupportedOperationException("Unsupported type:" + type);
}
}
}
Loading
Loading