Skip to content

Commit

Permalink
[FLINK-35647][route] Support symbol replacement to enrich routing rules
Browse files Browse the repository at this point in the history
This closes #3428.

Co-authored-by: 张田 <[email protected]>
Co-authored-by: yangshuaitong <[email protected]>
  • Loading branch information
3 people authored Jul 12, 2024
1 parent 302a691 commit ad1f554
Show file tree
Hide file tree
Showing 16 changed files with 494 additions and 80 deletions.
27 changes: 21 additions & 6 deletions docs/content.zh/docs/core-concept/route.md
Original file line number Diff line number Diff line change
Expand Up @@ -30,11 +30,12 @@ under the License.
# Parameters
To describe a route, the follows are required:

| parameter | meaning | optional/required |
|--------------|----------------------------------------------------|-------------------|
| source-table | Source table id, supports regular expressions | required |
| sink-table | Sink table id, supports regular expressions | required |
| description | Routing rule description(a default value provided) | optional |
| parameter | meaning | optional/required |
|----------------|---------------------------------------------------------------------------------------------|-------------------|
| source-table | Source table id, supports regular expressions | required |
| sink-table | Sink table id, supports symbol replacement | required |
| replace-symbol | Special symbol in sink-table for pattern replacing, will be replaced by original table name | optional |
| description | Routing rule description(a default value provided) | optional |

A route module can contain a list of source-table/sink-table rules.

Expand Down Expand Up @@ -71,4 +72,18 @@ route:
- source-table: mydb.products
sink-table: ods_db.ods_products
description: sync products table to ods_products
```
```

## Pattern Replacement in routing rules

If you'd like to route source tables and rename them to sink tables with specific patterns, `replace-symbol` could be used to resemble source table names like this:

```yaml
route:
- source-table: source_db.\.*
sink-table: sink_db.<>
replace-symbol: <>
description: route all tables in source_db to sink_db
```

Then, all tables including `source_db.XXX` will be routed to `sink_db.XXX` without hassle.
27 changes: 21 additions & 6 deletions docs/content/docs/core-concept/route.md
Original file line number Diff line number Diff line change
Expand Up @@ -30,11 +30,12 @@ under the License.
# Parameters
To describe a route, the follows are required:

| parameter | meaning | optional/required |
|--------------|----------------------------------------------------|-------------------|
| source-table | Source table id, supports regular expressions | required |
| sink-table | Sink table id, supports regular expressions | required |
| description | Routing rule description(a default value provided) | optional |
| parameter | meaning | optional/required |
|----------------|---------------------------------------------------------------------------------------------|-------------------|
| source-table | Source table id, supports regular expressions | required |
| sink-table | Sink table id, supports symbol replacement | required |
| replace-symbol | Special symbol in sink-table for pattern replacing, will be replaced by original table name | optional |
| description | Routing rule description(a default value provided) | optional |

A route module can contain a list of source-table/sink-table rules.

Expand Down Expand Up @@ -71,4 +72,18 @@ route:
- source-table: mydb.products
sink-table: ods_db.ods_products
description: sync products table to ods_products
```
```

## Pattern Replacement in routing rules

If you'd like to route source tables and rename them to sink tables with specific patterns, `replace-symbol` could be used to resemble source table names like this:

```yaml
route:
- source-table: source_db.\.*
sink-table: sink_db.<>
replace-symbol: <>
description: route all tables in source_db to sink_db
```

Then, all tables including `source_db.XXX` will be routed to `sink_db.XXX` without hassle.
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ public class YamlPipelineDefinitionParser implements PipelineDefinitionParser {
// Route keys
private static final String ROUTE_SOURCE_TABLE_KEY = "source-table";
private static final String ROUTE_SINK_TABLE_KEY = "sink-table";
private static final String ROUTE_REPLACE_SYMBOL = "replace-symbol";
private static final String ROUTE_DESCRIPTION_KEY = "description";

// Transform keys
Expand Down Expand Up @@ -164,11 +165,15 @@ private RouteDef toRouteDef(JsonNode routeNode) {
"Missing required field \"%s\" in route configuration",
ROUTE_SINK_TABLE_KEY)
.asText();
String replaceSymbol =
Optional.ofNullable(routeNode.get(ROUTE_REPLACE_SYMBOL))
.map(JsonNode::asText)
.orElse(null);
String description =
Optional.ofNullable(routeNode.get(ROUTE_DESCRIPTION_KEY))
.map(JsonNode::asText)
.orElse(null);
return new RouteDef(sourceTable, sinkTable, description);
return new RouteDef(sourceTable, sinkTable, replaceSymbol, description);
}

private TransformDef toTransformDef(JsonNode transformNode) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,6 +166,15 @@ void testInvalidTimeZone() throws Exception {
+ "Or use 'UTC' without time zone and daylight saving time.");
}

@Test
void testRouteWithReplacementSymbol() throws Exception {
URL resource =
Resources.getResource("definitions/pipeline-definition-full-with-repsym.yaml");
YamlPipelineDefinitionParser parser = new YamlPipelineDefinitionParser();
PipelineDef pipelineDef = parser.parse(Paths.get(resource.toURI()), new Configuration());
assertThat(pipelineDef).isEqualTo(fullDefWithRouteRepSym);
}

private final PipelineDef fullDef =
new PipelineDef(
new SourceDef(
Expand Down Expand Up @@ -197,10 +206,12 @@ void testInvalidTimeZone() throws Exception {
new RouteDef(
"mydb.default.app_order_.*",
"odsdb.default.app_order",
null,
"sync all sharding tables to one"),
new RouteDef(
"mydb.default.web_order",
"odsdb.default.ods_web_order",
null,
"sync table to with given prefix ods_")),
Arrays.asList(
new TransformDef(
Expand Down Expand Up @@ -258,10 +269,12 @@ void testInvalidTimeZone() throws Exception {
new RouteDef(
"mydb.default.app_order_.*",
"odsdb.default.app_order",
null,
"sync all sharding tables to one"),
new RouteDef(
"mydb.default.web_order",
"odsdb.default.ods_web_order",
null,
"sync table to with given prefix ods_")),
Arrays.asList(
new TransformDef(
Expand Down Expand Up @@ -312,7 +325,10 @@ void testInvalidTimeZone() throws Exception {
.build())),
Collections.singletonList(
new RouteDef(
"mydb.default.app_order_.*", "odsdb.default.app_order", null)),
"mydb.default.app_order_.*",
"odsdb.default.app_order",
null,
null)),
Collections.emptyList(),
Configuration.fromMap(
ImmutableMap.<String, String>builder()
Expand All @@ -326,4 +342,67 @@ void testInvalidTimeZone() throws Exception {
Collections.emptyList(),
Collections.emptyList(),
Configuration.fromMap(Collections.singletonMap("parallelism", "1")));

private final PipelineDef fullDefWithRouteRepSym =
new PipelineDef(
new SourceDef(
"mysql",
"source-database",
Configuration.fromMap(
ImmutableMap.<String, String>builder()
.put("host", "localhost")
.put("port", "3306")
.put("username", "admin")
.put("password", "pass")
.put(
"tables",
"adb.*, bdb.user_table_[0-9]+, [app|web]_order_.*")
.put(
"chunk-column",
"app_order_.*:id,web_order:product_id")
.put("capture-new-tables", "true")
.build())),
new SinkDef(
"kafka",
"sink-queue",
Configuration.fromMap(
ImmutableMap.<String, String>builder()
.put("bootstrap-servers", "localhost:9092")
.put("auto-create-table", "true")
.build())),
Arrays.asList(
new RouteDef(
"mydb.default.app_order_.*",
"odsdb.default.app_order_<>",
"<>",
"sync all sharding tables to one"),
new RouteDef(
"mydb.default.web_order",
"odsdb.default.ods_web_order_>_<",
">_<",
"sync table to with given prefix ods_")),
Arrays.asList(
new TransformDef(
"mydb.app_order_.*",
"id, order_id, TO_UPPER(product_name)",
"id > 10 AND order_id > 100",
"id",
"product_name",
"comment=app order",
"project fields from source table"),
new TransformDef(
"mydb.web_order_.*",
"CONCAT(id, order_id) as uniq_id, *",
"uniq_id > 10",
null,
null,
null,
"add new uniq_id for each row")),
Configuration.fromMap(
ImmutableMap.<String, String>builder()
.put("name", "source-database-sync-pipe")
.put("parallelism", "4")
.put("schema.change.behavior", "evolve")
.put("schema-operator.rpc-timeout", "1 h")
.build()));
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
################################################################################
# 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.
################################################################################
source:
type: mysql
name: source-database
host: localhost
port: 3306
username: admin
password: pass
tables: adb.*, bdb.user_table_[0-9]+, [app|web]_order_.*
chunk-column: app_order_.*:id,web_order:product_id
capture-new-tables: true

sink:
type: kafka
name: sink-queue
bootstrap-servers: localhost:9092
auto-create-table: true

route:
- source-table: mydb.default.app_order_.*
sink-table: odsdb.default.app_order_<>
replace-symbol: "<>"
description: sync all sharding tables to one
- source-table: mydb.default.web_order
sink-table: odsdb.default.ods_web_order_>_<
replace-symbol: ">_<"
description: sync table to with given prefix ods_

transform:
- source-table: mydb.app_order_.*
projection: id, order_id, TO_UPPER(product_name)
filter: id > 10 AND order_id > 100
primary-keys: id
partition-keys: product_name
table-options: comment=app order
description: project fields from source table
- source-table: mydb.web_order_.*
projection: CONCAT(id, order_id) as uniq_id, *
filter: uniq_id > 10
description: add new uniq_id for each row

pipeline:
name: source-database-sync-pipe
parallelism: 4
schema.change.behavior: evolve
schema-operator.rpc-timeout: 1 h
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.flink.cdc.common.route;

import java.io.Serializable;

/** Definition of a routing rule with replacement symbol. */
public class RouteRule implements Serializable {

private static final long serialVersionUID = 1L;

public RouteRule(String sourceTable, String sinkTable, String replaceSymbol) {
this.sourceTable = sourceTable;
this.sinkTable = sinkTable;
this.replaceSymbol = replaceSymbol;
}

public String sourceTable;
public String sinkTable;
public String replaceSymbol;
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,11 +36,17 @@
public class RouteDef {
private final String sourceTable;
private final String sinkTable;
private final String replaceSymbol;
@Nullable private final String description;

public RouteDef(String sourceTable, String sinkTable, @Nullable String description) {
public RouteDef(
String sourceTable,
String sinkTable,
@Nullable String replaceSymbol,
@Nullable String description) {
this.sourceTable = sourceTable;
this.sinkTable = sinkTable;
this.replaceSymbol = replaceSymbol;
this.description = description;
}

Expand All @@ -52,6 +58,10 @@ public String getSinkTable() {
return sinkTable;
}

public Optional<String> getReplaceSymbol() {
return Optional.ofNullable(replaceSymbol);
}

public Optional<String> getDescription() {
return Optional.ofNullable(description);
}
Expand All @@ -63,6 +73,8 @@ public String toString() {
+ sourceTable
+ ", sinkTable="
+ sinkTable
+ ", replaceSymbol="
+ replaceSymbol
+ ", description='"
+ description
+ '\''
Expand All @@ -80,11 +92,12 @@ public boolean equals(Object o) {
RouteDef routeDef = (RouteDef) o;
return Objects.equals(sourceTable, routeDef.sourceTable)
&& Objects.equals(sinkTable, routeDef.sinkTable)
&& Objects.equals(replaceSymbol, routeDef.replaceSymbol)
&& Objects.equals(description, routeDef.description);
}

@Override
public int hashCode() {
return Objects.hash(sourceTable, sinkTable, description);
return Objects.hash(sourceTable, sinkTable, replaceSymbol, description);
}
}
Loading

0 comments on commit ad1f554

Please sign in to comment.