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

[cdc-pipeline-connector][starrocks] Introduce starrocks cdc pipeline DataSink #2765

Closed
wants to merge 20 commits 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
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Copyright 2023 Ververica Inc.

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">
<parent>
<artifactId>flink-cdc-pipeline-connectors</artifactId>
<groupId>com.ververica</groupId>
<version>${revision}</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>flink-cdc-pipeline-connector-starrocks</artifactId>

<properties>
<starrocks.connector.version>1.2.9_flink-${flink.major.version}</starrocks.connector.version>
</properties>

<dependencies>
<dependency>
<groupId>com.starrocks</groupId>
<artifactId>flink-connector-starrocks</artifactId>
<version>${starrocks.connector.version}</version>
</dependency>

<dependency>
<!-- TODO connector 1.2.9 depends on this, but not package it, so add this dependency here.
This dependency can be removed after upgrading connector to 1.2.10 which will not use
commons-compress anymore. -->
<groupId>org.apache.commons</groupId>
<artifactId>commons-compress</artifactId>
<version>1.21</version>
</dependency>

<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-cdc-composer</artifactId>
<version>${revision}</version>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<version>3.1.1</version>
<executions>
<execution>
<id>shade-flink</id>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<shadeTestJar>false</shadeTestJar>
<artifactSet>
<includes>
<include>com.starrocks:*</include>
<include>org.apache.commons:commons-compress</include>
</includes>
</artifactSet>
<relocations>
<relocation>
<pattern>org.apache.commons.compress</pattern>
<shadedPattern>com.starrocks.shade.org.apache.commons.compress</shadedPattern>
</relocation>
</relocations>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,148 @@
/*
* Copyright 2023 Ververica Inc.
*
* 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.ververica.cdc.connectors.starrocks.sink;

import org.apache.flink.api.common.serialization.SerializationSchema;

import com.starrocks.connector.flink.table.data.DefaultStarRocksRowData;
import com.starrocks.connector.flink.table.data.StarRocksRowData;
import com.starrocks.connector.flink.table.sink.v2.RecordSerializationSchema;
import com.starrocks.connector.flink.table.sink.v2.StarRocksSinkContext;
import com.starrocks.connector.flink.tools.JsonWrapper;
import com.ververica.cdc.common.data.RecordData;
import com.ververica.cdc.common.event.CreateTableEvent;
import com.ververica.cdc.common.event.DataChangeEvent;
import com.ververica.cdc.common.event.Event;
import com.ververica.cdc.common.event.SchemaChangeEvent;
import com.ververica.cdc.common.event.TableId;
import com.ververica.cdc.common.schema.Column;
import com.ververica.cdc.common.schema.Schema;
import com.ververica.cdc.common.utils.Preconditions;
import com.ververica.cdc.common.utils.SchemaUtils;

import java.time.ZoneId;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

import static com.ververica.cdc.connectors.starrocks.sink.StarRocksUtils.createFieldGetter;

/** Serializer for the input {@link Event}. It will serialize a row to a json string. */
public class EventRecordSerializationSchema implements RecordSerializationSchema<Event> {

private static final long serialVersionUID = 1L;

/**
* The local time zone used when converting from <code>TIMESTAMP WITH LOCAL TIME ZONE</code>.
*/
private final ZoneId zoneId;

/** keep the relationship of TableId and table information. */
private transient Map<TableId, TableInfo> tableInfoMap;

private transient DefaultStarRocksRowData reusableRowData;
private transient JsonWrapper jsonWrapper;

public EventRecordSerializationSchema(ZoneId zoneId) {
this.zoneId = zoneId;
}

@Override
public void open(
SerializationSchema.InitializationContext context, StarRocksSinkContext sinkContext) {
this.tableInfoMap = new HashMap<>();
this.reusableRowData = new DefaultStarRocksRowData();
this.jsonWrapper = new JsonWrapper();
}

@Override
public StarRocksRowData serialize(Event record) {
if (record instanceof SchemaChangeEvent) {
applySchemaChangeEvent((SchemaChangeEvent) record);
return null;
Copy link
Contributor

@lvyanquan lvyanquan Nov 28, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just a worry, will this lead to NoPointException?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the reminding. Connector will deal with the null value.

} else if (record instanceof DataChangeEvent) {
return applyDataChangeEvent((DataChangeEvent) record);
} else {
throw new UnsupportedOperationException("Don't support event " + record);
}
}

private void applySchemaChangeEvent(SchemaChangeEvent event) {
TableId tableId = event.tableId();
Schema newSchema;
if (event instanceof CreateTableEvent) {
newSchema = ((CreateTableEvent) event).getSchema();
} else {
TableInfo tableInfo = tableInfoMap.get(tableId);
if (tableInfo == null) {
throw new RuntimeException("schema of " + tableId + " is not existed.");
}
newSchema = SchemaUtils.applySchemaChangeEvent(tableInfo.schema, event);
}
TableInfo tableInfo = new TableInfo();
tableInfo.schema = newSchema;
tableInfo.fieldGetters = new RecordData.FieldGetter[newSchema.getColumnCount()];
for (int i = 0; i < newSchema.getColumnCount(); i++) {
tableInfo.fieldGetters[i] =
createFieldGetter(newSchema.getColumns().get(i).getType(), i, zoneId);
}
tableInfoMap.put(tableId, tableInfo);
}

private StarRocksRowData applyDataChangeEvent(DataChangeEvent event) {
TableInfo tableInfo = tableInfoMap.get(event.tableId());
Preconditions.checkNotNull(tableInfo, event.tableId() + " is not existed");
reusableRowData.setDatabase(event.tableId().getSchemaName());
reusableRowData.setTable(event.tableId().getTableName());
String value;
switch (event.op()) {
case INSERT:
case UPDATE:
case REPLACE:
value = serializeRecord(tableInfo, event.after(), false);
break;
case DELETE:
value = serializeRecord(tableInfo, event.before(), true);
break;
default:
throw new UnsupportedOperationException(
"Don't support operation type " + event.op());
}
reusableRowData.setRow(value);
return reusableRowData;
}

private String serializeRecord(TableInfo tableInfo, RecordData record, boolean isDelete) {
List<Column> columns = tableInfo.schema.getColumns();
Preconditions.checkArgument(columns.size() == record.getArity());
Map<String, Object> rowMap = new HashMap<>(record.getArity() + 1);
for (int i = 0; i < record.getArity(); i++) {
rowMap.put(columns.get(i).getName(), tableInfo.fieldGetters[i].getFieldOrNull(record));
}
rowMap.put("__op", isDelete ? 1 : 0);
return jsonWrapper.toJSONString(rowMap);
}

@Override
public void close() {}

/** Table information. */
private static class TableInfo {
Schema schema;
RecordData.FieldGetter[] fieldGetters;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Copyright 2023 Ververica Inc.
*
* 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.ververica.cdc.connectors.starrocks.sink;

import com.ververica.cdc.common.configuration.Configuration;
import com.ververica.cdc.common.utils.Preconditions;

import java.io.Serializable;

import static com.ververica.cdc.connectors.starrocks.sink.StarRocksDataSinkOptions.TABLE_SCHEMA_CHANGE_TIMEOUT;

/** Configurations for schema change. */
public class SchemaChangeConfig implements Serializable {

private static final long serialVersionUID = 1L;

/** Timeout for a schema change on StarRocks side. */
private final long timeoutSecond;

public SchemaChangeConfig(long timeoutSecond) {
Preconditions.checkArgument(
timeoutSecond > 0, "Timeout must be positive, but actually is %s", timeoutSecond);
this.timeoutSecond = timeoutSecond;
}

public long getTimeoutSecond() {
return timeoutSecond;
}

public static SchemaChangeConfig from(Configuration config) {
long timeoutSecond = Math.max(1, config.get(TABLE_SCHEMA_CHANGE_TIMEOUT).getSeconds());
return new SchemaChangeConfig(timeoutSecond);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
/*
* Copyright 2023 Ververica Inc.
*
* 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.ververica.cdc.connectors.starrocks.sink;

import com.starrocks.connector.flink.catalog.StarRocksCatalog;
import com.starrocks.connector.flink.table.sink.SinkFunctionFactory;
import com.starrocks.connector.flink.table.sink.StarRocksSinkOptions;
import com.starrocks.connector.flink.table.sink.v2.StarRocksSink;
import com.ververica.cdc.common.event.Event;
import com.ververica.cdc.common.sink.DataSink;
import com.ververica.cdc.common.sink.EventSinkProvider;
import com.ververica.cdc.common.sink.FlinkSinkProvider;
import com.ververica.cdc.common.sink.MetadataApplier;

import java.io.Serializable;
import java.time.ZoneId;

/** A {@link DataSink} for StarRocks connector that supports schema evolution. */
public class StarRocksDataSink implements DataSink, Serializable {

private static final long serialVersionUID = 1L;

/** Configurations for sink connector. */
private final StarRocksSinkOptions sinkOptions;

/** Configurations for creating a StarRocks table. */
private final TableCreateConfig tableCreateConfig;

/** Configurations for schema change. */
private final SchemaChangeConfig schemaChangeConfig;

/**
* The local time zone used when converting from <code>TIMESTAMP WITH LOCAL TIME ZONE</code>.
*/
private final ZoneId zoneId;

public StarRocksDataSink(
StarRocksSinkOptions sinkOptions,
TableCreateConfig tableCreateConfig,
SchemaChangeConfig schemaChangeConfig,
ZoneId zoneId) {
this.sinkOptions = sinkOptions;
this.tableCreateConfig = tableCreateConfig;
this.schemaChangeConfig = schemaChangeConfig;
this.zoneId = zoneId;
}

@Override
public EventSinkProvider getEventSinkProvider() {
StarRocksSink<Event> starRocksSink =
SinkFunctionFactory.createSink(
sinkOptions, new EventRecordSerializationSchema(zoneId));
return FlinkSinkProvider.of(starRocksSink);
}

@Override
public MetadataApplier getMetadataApplier() {
StarRocksCatalog catalog =
new StarRocksCatalog(
sinkOptions.getJdbcUrl(),
sinkOptions.getUsername(),
sinkOptions.getPassword());
return new StarRocksMetadataApplier(catalog, tableCreateConfig, schemaChangeConfig);
}
}
Loading