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] Added pulsar database cdc synchronization action #2345

Merged
merged 13 commits into from
Nov 30, 2023
Prev Previous commit
Next Next commit
add abstract SyncDatabaseActionBase.
  • Loading branch information
zhuangchong committed Nov 17, 2023
commit 8ad26e72dfba266a4ec733764b0e5a096864dffe
Original file line number Diff line number Diff line change
Expand Up @@ -18,29 +18,15 @@

package org.apache.paimon.flink.action.cdc;

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.catalog.AbstractCatalog;
import org.apache.paimon.flink.action.Action;
import org.apache.paimon.flink.action.ActionBase;
import org.apache.paimon.flink.action.MultiTablesSinkMode;
import org.apache.paimon.flink.action.cdc.format.DataFormat;
import org.apache.paimon.flink.action.cdc.format.RecordParser;
import org.apache.paimon.flink.sink.cdc.EventParser;
import org.apache.paimon.flink.sink.cdc.FlinkCdcSyncDatabaseSinkBuilder;
import org.apache.paimon.flink.sink.cdc.NewTableSchemaBuilder;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecordEventParser;

import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.connector.source.Source;
import org.apache.flink.configuration.Configuration;

import javax.annotation.Nullable;
import org.apache.flink.api.common.functions.FlatMapFunction;

import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.regex.Pattern;

/**
* An {@link Action} which synchronize the Multiple message queue topics into one Paimon database.
Expand Down Expand Up @@ -74,130 +60,22 @@
* <p>To automatically synchronize new table, This action creates a single sink for all Paimon
* tables to be written. See {@link MultiTablesSinkMode#COMBINED}.
*/
public abstract class MessageQueueSyncDatabaseActionBase extends ActionBase {

protected final String database;
protected final Configuration cdcSourceConfig;

private Map<String, String> tableConfig = new HashMap<>();
private String tablePrefix = "";
private String tableSuffix = "";
private String includingTables = ".*";
@Nullable String excludingTables;
private TypeMapping typeMapping = TypeMapping.defaultMapping();
public abstract class MessageQueueSyncDatabaseActionBase extends SyncDatabaseActionBase {

public MessageQueueSyncDatabaseActionBase(
String warehouse,
String database,
Map<String, String> catalogConfig,
Map<String, String> kafkaConfig) {
super(warehouse, catalogConfig);
this.database = database;
this.cdcSourceConfig = Configuration.fromMap(kafkaConfig);
}

public MessageQueueSyncDatabaseActionBase withTableConfig(Map<String, String> tableConfig) {
this.tableConfig = tableConfig;
return this;
}

public MessageQueueSyncDatabaseActionBase withTablePrefix(@Nullable String tablePrefix) {
if (tablePrefix != null) {
this.tablePrefix = tablePrefix;
}
return this;
}

public MessageQueueSyncDatabaseActionBase withTableSuffix(@Nullable String tableSuffix) {
if (tableSuffix != null) {
this.tableSuffix = tableSuffix;
}
return this;
}

public MessageQueueSyncDatabaseActionBase includingTables(@Nullable String includingTables) {
if (includingTables != null) {
this.includingTables = includingTables;
}
return this;
}

public MessageQueueSyncDatabaseActionBase excludingTables(@Nullable String excludingTables) {
this.excludingTables = excludingTables;
return this;
}

public MessageQueueSyncDatabaseActionBase withTypeMapping(TypeMapping typeMapping) {
this.typeMapping = typeMapping;
return this;
Map<String, String> mqConfig) {
super(warehouse, database, catalogConfig, mqConfig);
}

@Override
public void build() throws Exception {
boolean caseSensitive = catalog.caseSensitive();

validateCaseInsensitive(caseSensitive);

catalog.createDatabase(database, true);

Source<String, ?, ?> source = buildSource();

protected FlatMapFunction<String, RichCdcMultiplexRecord> recordParse() {
DataFormat format = getDataFormat();
RecordParser recordParser =
format.createParser(caseSensitive, typeMapping, Collections.emptyList());
NewTableSchemaBuilder schemaBuilder = new NewTableSchemaBuilder(tableConfig, caseSensitive);
Pattern includingPattern = Pattern.compile(includingTables);
Pattern excludingPattern =
excludingTables == null ? null : Pattern.compile(excludingTables);
TableNameConverter tableNameConverter =
new TableNameConverter(caseSensitive, true, tablePrefix, tableSuffix);
EventParser.Factory<RichCdcMultiplexRecord> parserFactory =
() ->
new RichCdcMultiplexRecordEventParser(
schemaBuilder,
includingPattern,
excludingPattern,
tableNameConverter);

new FlinkCdcSyncDatabaseSinkBuilder<RichCdcMultiplexRecord>()
.withInput(
env.fromSource(source, WatermarkStrategy.noWatermarks(), sourceName())
.flatMap(recordParser)
.name("Parse"))
.withParserFactory(parserFactory)
.withCatalogLoader(catalogLoader())
.withDatabase(database)
.withMode(MultiTablesSinkMode.COMBINED)
.withTableOptions(tableConfig)
.build();
}

private void validateCaseInsensitive(boolean caseSensitive) {
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Database", database);
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Table prefix", tablePrefix);
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Table suffix", tableSuffix);
boolean caseSensitive = catalog.caseSensitive();
return format.createParser(caseSensitive, typeMapping, Collections.emptyList());
}

protected abstract Source<String, ?, ?> buildSource() throws Exception;

protected abstract DataFormat getDataFormat();

protected abstract String sourceName();

protected abstract String jobName();

@VisibleForTesting
public Map<String, String> tableConfig() {
return tableConfig;
}

// ------------------------------------------------------------------------
// Flink run methods
// ------------------------------------------------------------------------

@Override
public void run() throws Exception {
build();
execute(String.format("KAFKA-Paimon Database Sync: %s", database));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,206 @@
/*
* 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.apache.paimon.flink.action.cdc;

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.catalog.AbstractCatalog;
import org.apache.paimon.flink.action.Action;
import org.apache.paimon.flink.action.ActionBase;
import org.apache.paimon.flink.action.MultiTablesSinkMode;
import org.apache.paimon.flink.sink.cdc.EventParser;
import org.apache.paimon.flink.sink.cdc.FlinkCdcSyncDatabaseSinkBuilder;
import org.apache.paimon.flink.sink.cdc.NewTableSchemaBuilder;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecordEventParser;
import org.apache.paimon.table.FileStoreTable;

import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.connector.source.Source;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStream;

import javax.annotation.Nullable;

import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.regex.Pattern;

/** Base {@link Action} for synchronizing into one Paimon database. */
public abstract class SyncDatabaseActionBase extends ActionBase {

protected final String database;
protected final Configuration cdcSourceConfig;
protected Map<String, String> tableConfig = new HashMap<>();
protected String tablePrefix = "";
protected String tableSuffix = "";
protected String includingTables = ".*";
@Nullable protected Pattern excludingPattern;

protected TypeMapping typeMapping = TypeMapping.defaultMapping();

protected CdcMetadataConverter[] metadataConverters = new CdcMetadataConverter[] {};

public SyncDatabaseActionBase(
String warehouse,
String database,
Map<String, String> catalogConfig,
Map<String, String> cdcSourceConfig) {
super(warehouse, catalogConfig);
this.database = database;
this.cdcSourceConfig = Configuration.fromMap(cdcSourceConfig);
}

public SyncDatabaseActionBase withTableConfig(Map<String, String> tableConfig) {
this.tableConfig = tableConfig;
return this;
}

public SyncDatabaseActionBase withTablePrefix(@Nullable String tablePrefix) {
if (tablePrefix != null) {
this.tablePrefix = tablePrefix;
}
return this;
}

public SyncDatabaseActionBase withTableSuffix(@Nullable String tableSuffix) {
if (tableSuffix != null) {
this.tableSuffix = tableSuffix;
}
return this;
}

public SyncDatabaseActionBase includingTables(@Nullable String includingTables) {
if (includingTables != null) {
this.includingTables = includingTables;
}
return this;
}

public SyncDatabaseActionBase excludingTables(@Nullable String excludingTables) {
this.excludingPattern = excludingTables == null ? null : Pattern.compile(excludingTables);
return this;
}

public SyncDatabaseActionBase withTypeMapping(TypeMapping typeMapping) {
this.typeMapping = typeMapping;
return this;
}

public SyncDatabaseActionBase withMetadataColumns(List<String> metadataColumns) {
this.metadataConverters =
metadataColumns.stream()
.map(this::metadataConverter)
.filter(Optional::isPresent)
.map(Optional::get)
.toArray(CdcMetadataConverter[]::new);
return this;
}

protected Optional<CdcMetadataConverter<?>> metadataConverter(String column) {
return Optional.empty();
}

protected void checkCdcSourceArgument() {}

protected abstract Source<String, ?, ?> buildSource() throws Exception;

protected abstract String sourceName();

protected abstract FlatMapFunction<String, RichCdcMultiplexRecord> recordParse();

protected List<FileStoreTable> sinkTables() {
return Collections.emptyList();
}

protected MultiTablesSinkMode sinkMode() {
return MultiTablesSinkMode.COMBINED;
}

@Override
public void build() throws Exception {
checkCdcSourceArgument();
boolean caseSensitive = catalog.caseSensitive();

validateCaseInsensitive(caseSensitive);

catalog.createDatabase(database, true);

Source<String, ?, ?> source = buildSource();

NewTableSchemaBuilder schemaBuilder = new NewTableSchemaBuilder(tableConfig, caseSensitive);
Pattern includingPattern = Pattern.compile(includingTables);
TableNameConverter tableNameConverter =
new TableNameConverter(caseSensitive, true, tablePrefix, tableSuffix);

DataStream<RichCdcMultiplexRecord> input =
env.fromSource(source, WatermarkStrategy.noWatermarks(), sourceName())
.flatMap(recordParse())
.name("Parse");
EventParser.Factory<RichCdcMultiplexRecord> parserFactory =
() ->
new RichCdcMultiplexRecordEventParser(
schemaBuilder,
includingPattern,
excludingPattern,
tableNameConverter);

FlinkCdcSyncDatabaseSinkBuilder<RichCdcMultiplexRecord> sinkBuilder =
new FlinkCdcSyncDatabaseSinkBuilder<RichCdcMultiplexRecord>()
.withInput(input)
.withParserFactory(parserFactory)
.withCatalogLoader(catalogLoader())
.withDatabase(database)
.withMode(sinkMode())
.withTableOptions(tableConfig);

List<FileStoreTable> fileStoreTables = sinkTables();
if (!fileStoreTables.isEmpty()) {
sinkBuilder.withTables(sinkTables());
}

sinkBuilder.build();
}

protected void validateCaseInsensitive(boolean caseSensitive) {
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Database", database);
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Table prefix", tablePrefix);
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Table suffix", tableSuffix);
}

@VisibleForTesting
public Map<String, String> tableConfig() {
return tableConfig;
}

// ------------------------------------------------------------------------
// Flink run methods
// ------------------------------------------------------------------------

protected abstract String jobName();

@Override
public void run() throws Exception {
build();
execute(jobName());
}
}
Loading
Loading