-
Notifications
You must be signed in to change notification settings - Fork 866
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[cdc] Cdc sync normal json format to append table (#3389)
- Loading branch information
Showing
8 changed files
with
313 additions
and
3 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
65 changes: 65 additions & 0 deletions
65
...nk-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/json/JsonRecordParser.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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"; | ||
} | ||
} |
133 changes: 133 additions & 0 deletions
133
...rc/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaJsonSyncTableActionITCase.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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()); | ||
} | ||
} |
20 changes: 20 additions & 0 deletions
20
...flink/paimon-flink-cdc/src/test/resources/kafka/json/table/computedcolumn/json-data-1.txt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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"} |
20 changes: 20 additions & 0 deletions
20
...link/paimon-flink-cdc/src/test/resources/kafka/json/table/schemaevolution/json-data-1.txt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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"} |
20 changes: 20 additions & 0 deletions
20
...link/paimon-flink-cdc/src/test/resources/kafka/json/table/schemaevolution/json-data-2.txt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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"} |