forked from apache/paimon
-
Notifications
You must be signed in to change notification settings - Fork 2
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[flink][kafka-cdc] Support table debezium json format (apache#2251)
- Loading branch information
1 parent
c74987b
commit 4db2e9a
Showing
11 changed files
with
543 additions
and
13 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
143 changes: 143 additions & 0 deletions
143
...rc/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumRecordParser.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,143 @@ | ||
/* | ||
* 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.debezium; | ||
|
||
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; | ||
|
||
import static org.apache.flink.shaded.guava30.com.google.common.base.Preconditions.checkArgument; | ||
import static org.apache.paimon.utils.JsonSerdeUtil.isNull; | ||
|
||
/** | ||
* The {@code DebeziumRecordParser} class extends the abstract {@link RecordParser} and is designed | ||
* to parse records from Debezium's JSON change data capture (CDC) format. Debezium is a CDC | ||
* solution for MySQL databases that captures row-level changes to database tables and outputs them | ||
* in JSON format. This parser extracts relevant information from the Debezium-JSON format and | ||
* converts it into a list of {@link RichCdcMultiplexRecord} objects. | ||
* | ||
* <p>The class supports various database operations such as INSERT, UPDATE, DELETE, and READ | ||
* (snapshot reads), and creates corresponding {@link RichCdcMultiplexRecord} objects to represent | ||
* these changes. | ||
* | ||
* <p>Validation is performed to ensure that the JSON records contain all necessary fields, | ||
* including the 'before' and 'after' states for UPDATE operations, and the class also supports | ||
* schema extraction for the Kafka topic. Debezium's specific fields such as 'source', 'op' for | ||
* operation type, and primary key field names are used to construct the details of each record | ||
* event. | ||
*/ | ||
public class DebeziumRecordParser extends RecordParser { | ||
|
||
private static final String FIELD_BEFORE = "before"; | ||
private static final String FIELD_AFTER = "after"; | ||
private static final String FIELD_SOURCE = "source"; | ||
private static final String FIELD_PRIMARY = "pkNames"; | ||
private static final String FIELD_DB = "db"; | ||
private static final String FIELD_TYPE = "op"; | ||
private static final String OP_INSERT = "c"; | ||
private static final String OP_UPDATE = "u"; | ||
private static final String OP_DELETE = "d"; | ||
private static final String OP_READE = "r"; | ||
|
||
public DebeziumRecordParser( | ||
boolean caseSensitive, TypeMapping typeMapping, List<ComputedColumn> computedColumns) { | ||
super(caseSensitive, typeMapping, computedColumns); | ||
} | ||
|
||
@Override | ||
public List<RichCdcMultiplexRecord> extractRecords() { | ||
String operation = extractStringFromRootJson(FIELD_TYPE); | ||
List<RichCdcMultiplexRecord> records = new ArrayList<>(); | ||
switch (operation) { | ||
case OP_INSERT: | ||
case OP_READE: | ||
processRecord(root.get(dataField()), RowKind.INSERT, records); | ||
break; | ||
case OP_UPDATE: | ||
processRecord( | ||
mergeOldRecord(root.get(dataField()), root.get(FIELD_BEFORE)), | ||
RowKind.DELETE, | ||
records); | ||
processRecord(root.get(dataField()), RowKind.INSERT, records); | ||
break; | ||
case OP_DELETE: | ||
processRecord(root.get(FIELD_BEFORE), RowKind.DELETE, records); | ||
break; | ||
default: | ||
throw new UnsupportedOperationException("Unknown record operation: " + operation); | ||
} | ||
return records; | ||
} | ||
|
||
@Override | ||
protected void validateFormat() { | ||
String errorMessageTemplate = | ||
"Didn't find '%s' node in json. Please make sure your topic's format is correct."; | ||
checkArgument( | ||
!isNull(root.get(FIELD_SOURCE).get(FIELD_TABLE)), | ||
errorMessageTemplate, | ||
FIELD_TABLE); | ||
checkArgument( | ||
!isNull(root.get(FIELD_SOURCE).get(FIELD_DB)), | ||
errorMessageTemplate, | ||
FIELD_DATABASE); | ||
checkArgument(!isNull(root.get(FIELD_TYPE)), errorMessageTemplate, FIELD_TYPE); | ||
String operation = root.get(FIELD_TYPE).asText(); | ||
switch (operation) { | ||
case OP_INSERT: | ||
case OP_READE: | ||
checkArgument(!isNull(root.get(dataField())), errorMessageTemplate, dataField()); | ||
break; | ||
case OP_UPDATE: | ||
case OP_DELETE: | ||
checkArgument(!isNull(root.get(FIELD_BEFORE)), errorMessageTemplate, FIELD_BEFORE); | ||
break; | ||
default: | ||
throw new IllegalArgumentException("Unsupported operation type: " + operation); | ||
} | ||
checkArgument(!isNull(root.get(primaryField())), errorMessageTemplate, primaryField()); | ||
} | ||
|
||
@Override | ||
protected String primaryField() { | ||
return FIELD_PRIMARY; | ||
} | ||
|
||
@Override | ||
protected String dataField() { | ||
return FIELD_AFTER; | ||
} | ||
|
||
@Override | ||
protected String extractStringFromRootJson(String key) { | ||
if (key.equals(FIELD_TABLE)) { | ||
tableName = root.get(FIELD_SOURCE).get(FIELD_TABLE).asText(); | ||
return tableName; | ||
} else if (key.equals(FIELD_DATABASE)) { | ||
databaseName = root.get(FIELD_SOURCE).get(FIELD_DB).asText(); | ||
return databaseName; | ||
} | ||
return root.get(key) != null ? root.get(key).asText() : null; | ||
} | ||
} |
67 changes: 67 additions & 0 deletions
67
...g/apache/paimon/flink/action/cdc/format/debezium/JsonPrimaryKeyDeserializationSchema.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,67 @@ | ||
/* | ||
* 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.debezium; | ||
|
||
import org.apache.paimon.utils.JsonSerdeUtil; | ||
|
||
import org.apache.flink.api.common.serialization.DeserializationSchema; | ||
import org.apache.flink.api.common.typeinfo.BasicTypeInfo; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
|
||
import java.nio.charset.StandardCharsets; | ||
import java.util.List; | ||
|
||
import static org.apache.paimon.utils.Preconditions.checkNotNull; | ||
|
||
/** | ||
* This class is used to deserialize byte[] messages into String format, and then add primary key | ||
* fields to the JSON string. | ||
*/ | ||
public class JsonPrimaryKeyDeserializationSchema implements DeserializationSchema<String> { | ||
|
||
public static final String PRIMARY_KEY_NAMES = "pkNames"; | ||
private final List<String> primaryKeyNames; | ||
|
||
public JsonPrimaryKeyDeserializationSchema(List<String> primaryKeyNames) { | ||
this.primaryKeyNames = checkNotNull(primaryKeyNames); | ||
if (this.primaryKeyNames.isEmpty()) { | ||
throw new IllegalArgumentException("primary key must not be empty"); | ||
} | ||
} | ||
|
||
@Override | ||
public String deserialize(byte[] message) { | ||
try { | ||
String value = new String(message, StandardCharsets.UTF_8); | ||
return JsonSerdeUtil.putArrayToJsonString(value, PRIMARY_KEY_NAMES, primaryKeyNames); | ||
} catch (Exception e) { | ||
throw new RuntimeException("Failed to deserialize message", e); | ||
} | ||
} | ||
|
||
@Override | ||
public boolean isEndOfStream(String nextElement) { | ||
return false; | ||
} | ||
|
||
@Override | ||
public TypeInformation<String> getProducedType() { | ||
return BasicTypeInfo.STRING_TYPE_INFO; | ||
} | ||
} |
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
Oops, something went wrong.