Skip to content

Commit

Permalink
Extended functionality, corresponding to Issue 3633
Browse files Browse the repository at this point in the history
  • Loading branch information
李鹏 committed Aug 22, 2024
1 parent adb0279 commit b83ec27
Show file tree
Hide file tree
Showing 10 changed files with 143 additions and 1 deletion.
8 changes: 8 additions & 0 deletions docs/layouts/shortcodes/generated/kafka_sync_database.html
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,14 @@
<td>The partition keys for Paimon table. If there are multiple partition keys, connect them with comma, for example "dt,hh,mm".
If the keys are not in source table, the sink table won't set partition keys.</td>
</tr>
<tr>
<td><h5>--multiple_table_partition_keys</h5></td>
<td>The partition keys for each different Paimon table. If there are multiple partition keys, connect them with comma, for example
<li>--multiple_table_partition_keys tableName1=col1,col2.col3</li>
<li>--multiple_table_partition_keys tableName2=col4,col5.col6</li>
<li>--multiple_table_partition_keys tableName3=col7,col8.col9</li>
If the keys are not in source table, the sink table won't set partition keys.</td>
</tr>
<tr>
<td><h5>--primary_keys</h5></td>
<td>The primary keys for Paimon table. If there are multiple primary keys, connect them with comma, for example "buyer_id,seller_id".
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,4 +54,20 @@ public static void parseKeyValueString(Map<String, String> map, String kvString)
}
map.put(kv[0].trim(), kv[1].trim());
}

public static void parseKeyValueList(Map<String, List<String>> mapList, String kvString) {
String[] kv = kvString.split("=", 2);
if (kv.length != 2) {
throw new IllegalArgumentException(
String.format(
"Invalid key-value string '%s'. Please use format 'key=value'",
kvString));
}
String[] valueArr = kv[1].trim().split(",");
List<String> valueList = new ArrayList<>();
for (String value : valueArr) {
valueList.add(value);
}
mapList.put(kv[0].trim(), valueList);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ public class CdcActionCommonUtils {
public static final String PRIMARY_KEYS = "primary_keys";
public static final String COMPUTED_COLUMN = "computed_column";
public static final String METADATA_COLUMN = "metadata_column";
public static final String MULTIPLE_TABLE_PARTITION_KEYS = "multiple_table_partition_keys";

public static void assertSchemaCompatible(
TableSchema paimonSchema, List<DataField> sourceTableFields) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
Expand All @@ -56,6 +57,7 @@ public abstract class SyncDatabaseActionBase extends SynchronizationActionBase {
protected List<String> primaryKeys = new ArrayList<>();
@Nullable protected String excludingTables;
protected List<FileStoreTable> tables = new ArrayList<>();
protected Map<String, List<String>> partitionKeyMultiple = new HashMap<>();

public SyncDatabaseActionBase(
String warehouse,
Expand Down Expand Up @@ -130,6 +132,14 @@ protected FlatMapFunction<CdcSourceRecord, RichCdcMultiplexRecord> recordParse()
Collections.emptyList(), typeMapping, metadataConverters);
}

public SyncDatabaseActionBase withPartitionKeyMultiple(
Map<String, List<String>> partitionKeyMultiple) {
if (partitionKeyMultiple != null) {
this.partitionKeyMultiple = partitionKeyMultiple;
}
return this;
}

@Override
protected EventParser.Factory<RichCdcMultiplexRecord> buildEventParserFactory() {
NewTableSchemaBuilder schemaBuilder =
Expand All @@ -138,6 +148,7 @@ protected EventParser.Factory<RichCdcMultiplexRecord> buildEventParserFactory()
allowUpperCase,
partitionKeys,
primaryKeys,
partitionKeyMultiple,
metadataConverters);
Pattern includingPattern = Pattern.compile(includingTables);
Pattern excludingPattern =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@

import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.EXCLUDING_TABLES;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.INCLUDING_TABLES;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.MULTIPLE_TABLE_PARTITION_KEYS;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.PARTITION_KEYS;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.PRIMARY_KEYS;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_PREFIX;
Expand All @@ -52,6 +53,8 @@ protected void withParams(MultipleParameterToolAdapter params, T action) {
.withTableSuffix(params.get(TABLE_SUFFIX))
.includingTables(params.get(INCLUDING_TABLES))
.excludingTables(params.get(EXCLUDING_TABLES))
.withPartitionKeyMultiple(
optionalConfigMapList(params, MULTIPLE_TABLE_PARTITION_KEYS))
.withPartitionKeys();

if (params.has(PARTITION_KEYS)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,9 @@
import org.apache.paimon.schema.Schema;

import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
Expand All @@ -37,18 +39,21 @@ public class NewTableSchemaBuilder implements Serializable {
private final List<String> partitionKeys;
private final List<String> primaryKeys;
private final CdcMetadataConverter[] metadataConverters;
protected Map<String, List<String>> partitionKeyMultiple = new HashMap<>();

public NewTableSchemaBuilder(
Map<String, String> tableConfig,
boolean caseSensitive,
List<String> partitionKeys,
List<String> primaryKeys,
Map<String, List<String>> partitionKeyMultiple,
CdcMetadataConverter[] metadataConverters) {
this.tableConfig = tableConfig;
this.caseSensitive = caseSensitive;
this.metadataConverters = metadataConverters;
this.partitionKeys = partitionKeys;
this.primaryKeys = primaryKeys;
this.partitionKeyMultiple = partitionKeyMultiple;
}

public Optional<Schema> build(RichCdcMultiplexRecord record) {
Expand All @@ -59,10 +64,20 @@ public Optional<Schema> build(RichCdcMultiplexRecord record) {
record.primaryKeys(),
Collections.emptyMap(),
null);
List<String> specifiedPartitionKeys = new ArrayList<>();
if (partitionKeys != null && !partitionKeys.isEmpty()) {
specifiedPartitionKeys = partitionKeys;
} else {
List<String> partitionKeyMultipleList = partitionKeyMultiple.get(record.tableName());
if (partitionKeyMultipleList != null && !partitionKeyMultipleList.isEmpty()) {
specifiedPartitionKeys = partitionKeyMultipleList;
}
}

return Optional.of(
buildPaimonSchema(
record.tableName(),
partitionKeys,
specifiedPartitionKeys,
primaryKeys,
Collections.emptyList(),
tableConfig,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

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

import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.flink.action.ActionBase;
import org.apache.paimon.flink.action.ActionITCaseBase;
import org.apache.paimon.flink.action.cdc.kafka.KafkaSyncDatabaseActionFactory;
Expand All @@ -30,6 +31,7 @@
import org.apache.paimon.flink.action.cdc.pulsar.PulsarSyncDatabaseActionFactory;
import org.apache.paimon.flink.action.cdc.pulsar.PulsarSyncTableActionFactory;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.Table;
import org.apache.paimon.table.source.ReadBuilder;
import org.apache.paimon.table.source.TableScan;
import org.apache.paimon.types.DataField;
Expand All @@ -51,7 +53,9 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Collectors;

import static org.assertj.core.api.Assertions.assertThat;

Expand Down Expand Up @@ -110,6 +114,18 @@ protected void assertTableNotExists(String... tableNames) throws Exception {
assertThat(catalog.listTables(database)).doesNotContain(tableNames);
}

protected void assertTablePartitionKeys(Map<String, String> partitionKeyMultiple)
throws Exception {
// get All tableNames;
Set<String> tableNames = partitionKeyMultiple.keySet();
for (String tableName : tableNames) {
Table table = catalog.getTable(new Identifier(database, tableName));
String actual = table.partitionKeys().stream().collect(Collectors.joining(","));
String expected = partitionKeyMultiple.get(tableName);
assertThat(actual).isEqualTo(expected);
}
}

protected void waitForResult(
List<String> expected, FileStoreTable table, RowType rowType, List<String> primaryKeys)
throws Exception {
Expand Down Expand Up @@ -366,6 +382,7 @@ protected abstract class SyncDatabaseActionBuilder<T extends SynchronizationActi
private final List<String> partitionKeys = new ArrayList<>();
private final List<String> primaryKeys = new ArrayList<>();
private final List<String> metadataColumn = new ArrayList<>();
protected Map<String, String> partitionKeyMultiple = new HashMap<>();

public SyncDatabaseActionBuilder(Class<T> clazz, Map<String, String> sourceConfig) {
this.clazz = clazz;
Expand Down Expand Up @@ -437,6 +454,14 @@ public SyncDatabaseActionBuilder<T> withMetadataColumn(List<String> metadataColu
return this;
}

public SyncDatabaseActionBuilder<T> withPartitionKeyMultiple(
Map<String, String> partitionKeyMultiple) {
if (partitionKeyMultiple != null) {
this.partitionKeyMultiple = partitionKeyMultiple;
}
return this;
}

public T build() {
List<String> args =
new ArrayList<>(
Expand All @@ -461,6 +486,7 @@ public T build() {

args.addAll(listToArgs("--type-mapping", typeMappingModes));
args.addAll(listToArgs("--partition-keys", partitionKeys));
args.addAll(mapToArgs("--multiple-table-partition-keys", partitionKeyMultiple));
args.addAll(listToArgs("--primary-keys", primaryKeys));
args.addAll(listToArgs("--metadata-column", metadataColumn));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,10 @@

import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
Expand Down Expand Up @@ -640,4 +642,29 @@ public void testSpecifyKeys() throws Exception {
rowType2,
Collections.singletonList("k"));
}

@Test
@Timeout(60)
public void testMultipleTablePartitionKeys() throws Exception {
final String topic = "multiple-table-partition-keys";
createTestTopic(topic, 1, 1);
writeRecordsToKafka(
topic, "kafka/canal/database/multipletablepartitionkeys/canal-data-1.txt");
Map<String, String> kafkaConfig = getBasicKafkaConfig();
kafkaConfig.put(VALUE_FORMAT.key(), "canal-json");
kafkaConfig.put(TOPIC.key(), topic);
Map<String, String> partitionKeyMultiple = new HashMap<>();
partitionKeyMultiple.put("tt_1", "k1,k2");
partitionKeyMultiple.put("tt_2", "k1,k3");
KafkaSyncDatabaseAction action =
syncDatabaseActionBuilder(kafkaConfig)
.withTableConfig(getBasicTableConfig())
.withPartitionKeyMultiple(partitionKeyMultiple)
.build();
runActionWithDefaultEnv(action);
// check paimon tables
List<String> tableNames = new ArrayList<>(partitionKeyMultiple.keySet());
waitingTables(tableNames);
assertTablePartitionKeys(partitionKeyMultiple);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
/*
* 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.
*/

{"data":[{"k1":"1","k2":"2","k3":"3","v0":"one"}],"database":"paimon_sync_database_affix","es":1684770066000,"id":76,"isDdl":false,"mysqlType":{"k1":"INT","k2":"INT","k3":"INT","v0":"VARCHAR(10)"},"old":null,"pkNames":["k1","k2","k3"],"sql":"","sqlType":{"k1":4,"k2":4,"k3":4,"v0":12},"table":"tt_1","ts":1684770066165,"type":"INSERT"}
{"data":[{"k1":"1","k2":"2","k3":"3","v0":"three"}],"database":"paimon_sync_database_affix","es":1684770066000,"id":78,"isDdl":false,"mysqlType":{"k1":"INT","k2":"INT","k3":"INT","v0":"VARCHAR(10)"},"old":null,"pkNames":["k1","k2","k3"],"sql":"","sqlType":{"k1":4,"k2":4,"k3":4,"v0":12},"table":"tt_2","ts":1684770066821,"type":"INSERT"}

Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import java.util.Optional;

import static org.apache.paimon.utils.ParameterUtils.parseCommaSeparatedKeyValues;
import static org.apache.paimon.utils.ParameterUtils.parseKeyValueList;
import static org.apache.paimon.utils.ParameterUtils.parseKeyValueString;

/** Factory to create {@link Action}. */
Expand Down Expand Up @@ -164,4 +165,17 @@ default String getRequiredValue(MultipleParameterToolAdapter params, String key)
checkRequiredArgument(params, key);
return params.get(key);
}

default Map<String, List<String>> optionalConfigMapList(
MultipleParameterToolAdapter params, String key) {
if (!params.has(key)) {
return Collections.emptyMap();
}

Map<String, List<String>> config = new HashMap<>();
for (String kvString : params.getMultiParameter(key)) {
parseKeyValueList(config, kvString);
}
return config;
}
}

0 comments on commit b83ec27

Please sign in to comment.