Skip to content

Commit

Permalink
[cdc] Cdc sync normal json format to append table (#3389)
Browse files Browse the repository at this point in the history
  • Loading branch information
MOBIN-F committed May 28, 2024
1 parent 2164f66 commit 73a1e39
Show file tree
Hide file tree
Showing 8 changed files with 313 additions and 3 deletions.
27 changes: 26 additions & 1 deletion docs/content/flink/cdc-ingestion/kafka-cdc.md
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ flink-sql-connector-kafka-*.jar
```

## Supported Formats
Flink provides several Kafka CDC formats: Canal, Debezium, Ogg and Maxwell JSON.
Flink provides several Kafka CDC formats: Canal, Debezium, Ogg, Maxwell and Normal JSON.
If a message in a Kafka topic is a change event captured from another database using the Change Data Capture (CDC) tool, then you can use the Paimon Kafka CDC. Write the INSERT, UPDATE, DELETE messages parsed into the paimon table.
<table class="table table-bordered">
<thead>
Expand All @@ -59,6 +59,10 @@ If a message in a Kafka topic is a change event captured from another database u
<td><a href="https://nightlies.apache.org/flink/flink-docs-stable/docs/connectors/table/formats/ogg/">OGG CDC</a></td>
<td>True</td>
</tr>
<tr>
<td><a href="https://nightlies.apache.org/flink/flink-docs-stable/docs/connectors/table/formats/json/">JSON</a></td>
<td>True</td>
</tr>
</tbody>
</table>

Expand Down Expand Up @@ -157,6 +161,27 @@ Then you can submit synchronization job:
... (other conf)
```
Example 3:
For some append data (such as log data), it can be treated as special CDC data with only INSERT operation type, so you can use 'format=json' to synchronize such data to the Paimon table.
```bash
<FLINK_HOME>/bin/flink run \
/path/to/paimon-flink-action-{{< version >}}.jar \
kafka_sync_table \
--warehouse hdfs:///path/to/warehouse \
--database test_db \
--table test_table \
--partition_keys pt \
--computed_column 'pt=date_format(event_tm, yyyyMMdd)' \
--kafka_conf properties.bootstrap.servers=127.0.0.1:9020 \
--kafka_conf topic=test_log \
--kafka_conf properties.group.id=123456 \
--kafka_conf value.format=json \
--catalog_conf metastore=hive \
--catalog_conf uri=thrift://hive-metastore:9083 \
--table_conf sink.parallelism=4
```
## Synchronizing Databases
By using [KafkaSyncDatabaseAction](/docs/{{< param Branch >}}/api/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction) in a Flink DataStream job or directly through `flink run`, users can synchronize the multi topic or one topic into one Paimon database.
Expand Down
27 changes: 26 additions & 1 deletion docs/content/flink/cdc-ingestion/pulsar-cdc.md
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ flink-connector-pulsar-*.jar
```

## Supported Formats
Flink provides several Pulsar CDC formats: Canal, Debezium, Ogg and Maxwell JSON.
Flink provides several Pulsar CDC formats: Canal, Debezium, Ogg, Maxwell and Normal JSON.
If a message in a pulsar topic is a change event captured from another database using the Change Data Capture (CDC) tool, then you can use the Paimon Pulsar CDC. Write the INSERT, UPDATE, DELETE messages parsed into the paimon table.
<table class="table table-bordered">
<thead>
Expand All @@ -59,6 +59,10 @@ If a message in a pulsar topic is a change event captured from another database
<td><a href="https://nightlies.apache.org/flink/flink-docs-stable/docs/connectors/table/formats/ogg/">OGG CDC</a></td>
<td>True</td>
</tr>
<tr>
<td><a href="https://nightlies.apache.org/flink/flink-docs-stable/docs/connectors/table/formats/json/">JSON</a></td>
<td>True</td>
</tr>
</tbody>
</table>

Expand Down Expand Up @@ -156,6 +160,27 @@ Then you can submit synchronization job:
... (other conf)
```
Example 3:
For some append data (such as log data), it can be treated as special CDC data with only INSERT operation type, so you can use 'format=json' to synchronize such data to the Paimon table.
```bash
<FLINK_HOME>/bin/flink run \
/path/to/paimon-flink-action-{{< version >}}.jar \
kafka_sync_table \
--warehouse hdfs:///path/to/warehouse \
--database test_db \
--table test_table \
--partition_keys pt \
--computed_column 'pt=date_format(event_tm, yyyyMMdd)' \
--kafka_conf properties.bootstrap.servers=127.0.0.1:9020 \
--kafka_conf topic=test_log \
--kafka_conf properties.group.id=123456 \
--kafka_conf value.format=json \
--catalog_conf metastore=hive \
--catalog_conf uri=thrift://hive-metastore:9083 \
--table_conf sink.parallelism=4
```
## Synchronizing Databases
By using [PulsarSyncDatabaseAction](/docs/{{< param Branch >}}/api/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseAction) in a Flink DataStream job or directly through `flink run`, users can synchronize the multi topic or one topic into one Paimon database.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.paimon.flink.action.cdc.TypeMapping;
import org.apache.paimon.flink.action.cdc.format.canal.CanalRecordParser;
import org.apache.paimon.flink.action.cdc.format.debezium.DebeziumRecordParser;
import org.apache.paimon.flink.action.cdc.format.json.JsonRecordParser;
import org.apache.paimon.flink.action.cdc.format.maxwell.MaxwellRecordParser;
import org.apache.paimon.flink.action.cdc.format.ogg.OggRecordParser;

Expand All @@ -38,7 +39,8 @@ public enum DataFormat {
CANAL_JSON(CanalRecordParser::new),
OGG_JSON(OggRecordParser::new),
MAXWELL_JSON(MaxwellRecordParser::new),
DEBEZIUM_JSON(DebeziumRecordParser::new);
DEBEZIUM_JSON(DebeziumRecordParser::new),
JSON(JsonRecordParser::new);
// Add more data formats here if needed

private final RecordParserFactory parser;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* 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.format.json;

import org.apache.paimon.flink.action.cdc.ComputedColumn;
import org.apache.paimon.flink.action.cdc.TypeMapping;
import org.apache.paimon.flink.action.cdc.format.RecordParser;
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord;
import org.apache.paimon.types.RowKind;

import java.util.ArrayList;
import java.util.List;

/**
* The {@code JsonRecordParser} class extends the abstract {@link RecordParser} class and is
* designed to parse JSON records.
*
* <p>This class treats JSON records as special CDC data with only insert operation type and
* generates {@link RichCdcMultiplexRecord} objects with only INSERT operation types.
*/
public class JsonRecordParser extends RecordParser {

public JsonRecordParser(
boolean caseSensitive, TypeMapping typeMapping, List<ComputedColumn> computedColumns) {
super(caseSensitive, typeMapping, computedColumns);
}

@Override
protected List<RichCdcMultiplexRecord> extractRecords() {
List<RichCdcMultiplexRecord> records = new ArrayList<>();
processRecord(root, RowKind.INSERT, records);
return records;
}

@Override
protected String primaryField() {
return null;
}

@Override
protected String dataField() {
return null;
}

@Override
protected String format() {
return "json";
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,133 @@
/*
* 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.kafka;

import org.apache.paimon.types.DataType;
import org.apache.paimon.types.DataTypes;
import org.apache.paimon.types.RowType;

import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;

import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;

import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC;
import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT;

/** IT cases for {@link KafkaSyncTableAction}. */
public class KafkaJsonSyncTableActionITCase extends KafkaSyncTableActionITCase {

@Test
@Timeout(60)
public void testSchemaEvolution() throws Exception {
String topic = "schema-evolution";
Map<String, String> tableOptions = new HashMap<>();

createTestTopic(topic, 1, 1);
writeRecordsToKafka(topic, "kafka/json/table/schemaevolution/json-data-1.txt");

Map<String, String> kafkaConfig = getBasicKafkaConfig();
kafkaConfig.put(VALUE_FORMAT.key(), "json");
kafkaConfig.put(TOPIC.key(), topic);
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withTableConfig(getBasicTableConfig())
.withTableConfig(tableOptions)
.build();

runActionWithDefaultEnv(action);

RowType rowType =
RowType.of(
new DataType[] {DataTypes.STRING(), DataTypes.STRING(), DataTypes.STRING()},
new String[] {"a", "b", "event_tm"});

waitForResult(
Arrays.asList("+I[a1, b1, 2024-05-22 09:50:40]", "+I[a2, b2, 2024-05-23 10:20:56]"),
getFileStoreTable(tableName),
rowType,
Collections.emptyList());

writeRecordsToKafka(
topic, "kafka/json/table/schemaevolution/json-data-2.txt", "schemaevolution");

rowType =
RowType.of(
new DataType[] {
DataTypes.STRING(),
DataTypes.STRING(),
DataTypes.STRING(),
DataTypes.STRING()
},
new String[] {"a", "b", "event_tm", "c"});

waitForResult(
Arrays.asList(
"+I[a1, b1, 2024-05-22 09:50:40, NULL]",
"+I[a2, b2, 2024-05-23 10:20:56, NULL]",
"+I[a3, b3, 2024-05-22 19:50:40, NULL]",
"+I[a4, b4, 2024-05-23 15:20:56, c4]"),
getFileStoreTable(tableName),
rowType,
Collections.emptyList());
}

@Test
@Timeout(60)
public void testComputedColumn() throws Exception {
String topic = "computed_column";
Map<String, String> tableOptions = new HashMap<>();

createTestTopic(topic, 1, 1);
writeRecordsToKafka(topic, "kafka/json/table/computedcolumn/json-data-1.txt");

Map<String, String> kafkaConfig = getBasicKafkaConfig();
kafkaConfig.put(VALUE_FORMAT.key(), "json");
kafkaConfig.put(TOPIC.key(), topic);
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withTableConfig(getBasicTableConfig())
.withPartitionKeys("pt")
.withTableConfig(tableOptions)
.withComputedColumnArgs("pt=date_format(event_tm, yyyyMMdd)")
.build();

runActionWithDefaultEnv(action);

RowType rowType =
RowType.of(
new DataType[] {
DataTypes.STRING(),
DataTypes.STRING(),
DataTypes.STRING(),
DataTypes.STRING()
},
new String[] {"a", "b", "event_tm", "pt"});
waitForResult(
Arrays.asList(
"+I[a1, b1, 2024-05-20 20:50:30, 20240520]",
"+I[a2, b2, 2024-05-21 18:10:46, 20240521]"),
getFileStoreTable(tableName),
rowType,
Collections.emptyList());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* 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.
*/

{"a":"a1","b":"b1","event_tm":"2024-05-20 20:50:30"}
{"a":"a2","b":"b2","event_tm":"2024-05-21 18:10:46"}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* 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.
*/

{"a":"a1","b":"b1","event_tm":"2024-05-22 09:50:40"}
{"a":"a2","b":"b2","event_tm":"2024-05-23 10:20:56"}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* 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.
*/

{"a":"a3","b":"b3","event_tm":"2024-05-22 19:50:40"}
{"a":"a4","b":"b4","c":"c4","event_tm":"2024-05-23 15:20:56"}

0 comments on commit 73a1e39

Please sign in to comment.