Skip to content
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 @@ -23,6 +23,7 @@
import org.dinky.cdc.AbstractSinkBuilder;
import org.dinky.cdc.convert.DataTypeConverter;
import org.dinky.cdc.utils.FlinkStatementUtil;
import org.dinky.data.flink.table.FlinkTableObjectIdentifier;
import org.dinky.data.model.Column;
import org.dinky.data.model.FlinkCDCConfig;
import org.dinky.data.model.Schema;
Expand Down Expand Up @@ -54,6 +55,8 @@
import java.util.Map;
import java.util.stream.Collectors;

import cn.hutool.core.util.StrUtil;

public abstract class AbstractSqlSinkBuilder extends AbstractSinkBuilder implements Serializable {

protected AbstractSqlSinkBuilder() {}
Expand Down Expand Up @@ -220,19 +223,20 @@ protected void executeCatalogStatement() {}
* @return view name
*/
public static String replaceViewNameMiddleLineToUnderLine(String viewName) {
if (!viewName.isEmpty() && viewName.contains("-")) {
if (StrUtil.isNotBlank(viewName) && viewName.contains("-")) {
logger.warn("the view name [{}] contains '-', replace '-' to '_' for flink use view name", viewName);
return viewName.replaceAll("-", "_");
}
return viewName;
}

protected List<Operation> createInsertOperations(Table table, String viewName, String tableName) {
String cdcSqlInsert = FlinkStatementUtil.getCDCInsertSql(table, tableName, viewName, config);
protected List<Operation> createInsertOperations(
Table table, FlinkTableObjectIdentifier sourceTable, FlinkTableObjectIdentifier targetTable) {
String cdcSqlInsert = FlinkStatementUtil.getCDCInsertSql(table, targetTable, sourceTable, config);
logger.info(cdcSqlInsert);

List<Operation> operations = customTableEnvironment.getParser().parse(cdcSqlInsert);
logger.info("Create {} FlinkSQL insert into successful...", tableName);
logger.info("Create {} FlinkSQL insert into successful...", targetTable);
if (operations.isEmpty()) {
return operations;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import org.dinky.cdc.SinkBuilder;
import org.dinky.cdc.utils.FlinkStatementUtil;
import org.dinky.data.flink.table.FlinkTableObjectIdentifier;
import org.dinky.data.model.FlinkCDCConfig;
import org.dinky.data.model.Table;

Expand All @@ -44,47 +45,51 @@ private SQLSinkBuilder(FlinkCDCConfig config) {
super(config);
}

private String addSourceTableView(DataStream<Row> rowDataDataStream, Table table) {
private FlinkTableObjectIdentifier addSourceTableView(DataStream<Row> rowDataDataStream, Table table) {
// Because the name of the view on Flink is not allowed to have -, it needs to be replaced with - here_
String viewName = replaceViewNameMiddleLineToUnderLine("VIEW_" + table.getSchemaTableNameWithUnderline());

customTableEnvironment.createTemporaryView(
viewName, customTableEnvironment.fromChangelogStream(rowDataDataStream));
logger.info("Create {} temporaryView successful...", viewName);
return viewName;
return FlinkTableObjectIdentifier.of(viewName);
}

@Override
protected void addTableSink(DataStream<Row> rowDataDataStream, Table table) {
final String viewName = addSourceTableView(rowDataDataStream, table);
final FlinkTableObjectIdentifier viewName = addSourceTableView(rowDataDataStream, table);
final String sinkSchemaName = getSinkSchemaName(table);
final String sinkTableName = getSinkTableName(table);
final FlinkTableObjectIdentifier sinkTable = FlinkTableObjectIdentifier.of(sinkTableName);

// Multiple sinks and single sink
if (CollectionUtils.isEmpty(config.getSinks())) {
addSinkInsert(table, viewName, sinkTableName, sinkSchemaName, sinkTableName);
addSinkInsert(table, viewName, sinkTable, sinkSchemaName, sinkTable);
} else {
for (int index = 0; index < config.getSinks().size(); index++) {
String tableName = sinkTableName;
FlinkTableObjectIdentifier newSinkTable = sinkTable;
if (config.getSinks().size() != 1) {
tableName = sinkTableName + "_" + index;
newSinkTable = FlinkTableObjectIdentifier.of(sinkTable + "_" + index);
}

config.setSink(config.getSinks().get(index));
addSinkInsert(table, viewName, tableName, sinkSchemaName, sinkTableName);
addSinkInsert(table, viewName, newSinkTable, sinkSchemaName, sinkTable);
}
}
}

private List<Operation> addSinkInsert(
Table table, String viewName, String tableName, String sinkSchemaName, String sinkTableName) {
Table table,
FlinkTableObjectIdentifier sourceTable,
FlinkTableObjectIdentifier targetTable,
String sinkSchemaName,
FlinkTableObjectIdentifier sinkTable) {
String pkList = StringUtils.join(getPKList(table), ".");
String flinkDDL =
FlinkStatementUtil.getFlinkDDL(table, tableName, config, sinkSchemaName, sinkTableName, pkList);
String flinkDDL = FlinkStatementUtil.getFlinkDDL(table, targetTable, config, sinkSchemaName, sinkTable, pkList);
logger.info(flinkDDL);
customTableEnvironment.executeSql(flinkDDL);
logger.info("Create {} FlinkSQL DDL successful...", tableName);
return createInsertOperations(table, viewName, tableName);
logger.info("Create {} FlinkSQL DDL successful...", targetTable);
return createInsertOperations(table, sourceTable, targetTable);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.dinky.cdc.SinkBuilder;
import org.dinky.cdc.sql.AbstractSqlSinkBuilder;
import org.dinky.cdc.utils.FlinkStatementUtil;
import org.dinky.data.flink.table.FlinkTableObjectIdentifier;
import org.dinky.data.model.FlinkCDCConfig;
import org.dinky.data.model.Table;

Expand All @@ -46,15 +47,17 @@ public void addTableSink(DataStream<Row> rowDataDataStream, Table table) {
String catalogName = config.getSink().get("catalog.name");
String sinkSchemaName = getSinkSchemaName(table);
String tableName = getSinkTableName(table);
String sinkTableName = catalogName + ".`" + sinkSchemaName + "`.`" + tableName + "`";
// Because the name of the view on Flink is not allowed to have -, it needs to be replaced with - here_
String viewName = replaceViewNameMiddleLineToUnderLine("VIEW_" + table.getSchemaTableNameWithUnderline());

customTableEnvironment.createTemporaryView(
viewName, customTableEnvironment.fromChangelogStream(rowDataDataStream));
logger.info("Create {} temporaryView successful...", viewName);

createInsertOperations(table, viewName, sinkTableName);
createInsertOperations(
table,
FlinkTableObjectIdentifier.of(viewName),
FlinkTableObjectIdentifier.of(catalogName, sinkSchemaName, tableName));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.dinky.cdc.utils;

import org.dinky.data.flink.table.FlinkTableObjectIdentifier;
import org.dinky.data.model.Column;
import org.dinky.data.model.FlinkCDCConfig;
import org.dinky.data.model.Table;
Expand All @@ -34,9 +35,13 @@ public class FlinkStatementUtil {

private FlinkStatementUtil() {}

public static String getCDCInsertSql(Table table, String targetName, String sourceName, FlinkCDCConfig config) {
public static String getCDCInsertSql(
Table table,
FlinkTableObjectIdentifier targetTable,
FlinkTableObjectIdentifier sourceTable,
FlinkCDCConfig config) {
StringBuilder sb = new StringBuilder("INSERT INTO ");
sb.append(targetName);
sb.append(targetTable.toTablePath());
sb.append(" SELECT\n");
for (int i = 0; i < table.getColumns().size(); i++) {
sb.append(" ");
Expand All @@ -45,9 +50,8 @@ public static String getCDCInsertSql(Table table, String targetName, String sour
}
sb.append(getColumnProcessing(table.getColumns().get(i), config)).append(" \n");
}
sb.append(" FROM `");
sb.append(sourceName);
sb.append("`");
sb.append(" FROM ");
sb.append(sourceTable.toTablePath());
return sb.toString();
}

Expand All @@ -65,19 +69,19 @@ public static String getColumnProcessing(Column column, FlinkCDCConfig config) {

public static String getFlinkDDL(
Table table,
String tableName,
FlinkTableObjectIdentifier flinkTable,
FlinkCDCConfig config,
String sinkSchemaName,
String sinkTableName,
FlinkTableObjectIdentifier sinkTableName,
String pkList) {
StringBuilder sb = new StringBuilder();
if (Integer.parseInt(EnvironmentInformation.getVersion().split("\\.")[1]) < 13) {
sb.append("CREATE TABLE `");
sb.append("CREATE TABLE ");
} else {
sb.append("CREATE TABLE IF NOT EXISTS `");
sb.append("CREATE TABLE IF NOT EXISTS ");
}
sb.append(tableName);
sb.append("` (\n");
sb.append(flinkTable.toTablePath());
sb.append(" (\n");
List<String> pks = new ArrayList<>();
for (int i = 0; i < table.getColumns().size(); i++) {
String type = table.getColumns().get(i).getFlinkType();
Expand Down Expand Up @@ -109,7 +113,7 @@ public static String getFlinkDDL(
sb.append(pksb);
}
sb.append(") WITH (\n");
sb.append(getSinkConfigurationString(config, sinkSchemaName, sinkTableName, pkList));
sb.append(getSinkConfigurationString(config, sinkSchemaName, sinkTableName.getObjectName(), pkList));
sb.append(")\n");
return sb.toString();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
*
* 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.dinky.data.flink.table;

import lombok.Getter;

@Getter
public class FlinkTableObjectIdentifier {
private final String catalogName;
private final String databaseName;
private final String objectName;

public static FlinkTableObjectIdentifier of(String catalogName, String databaseName, String objectName) {
return new FlinkTableObjectIdentifier(catalogName, databaseName, objectName);
}

public static FlinkTableObjectIdentifier of(String objectName) {
return of(null, null, objectName);
}

private FlinkTableObjectIdentifier(String catalogName, String databaseName, String objectName) {
this.catalogName = catalogName;
this.databaseName = databaseName;
this.objectName = objectName;
if (objectName == null) {
throw new IllegalArgumentException("objectName can not be null");
}
}

/**
*
* @return catalogName.`databaseName`.`objectName`
*/
public String toTablePath() {
StringBuilder sb = new StringBuilder();
if (catalogName != null) {
sb.append(catalogName);
}
if (databaseName != null) {
sb.append(".`");
sb.append(databaseName);
sb.append("`.");
}
sb.append("`");
sb.append(objectName);
sb.append("`");
return sb.toString();
}

@Override
public String toString() {
return toTablePath();
}
}