-
Notifications
You must be signed in to change notification settings - Fork 1.3k
[flink] Supports debezium-bson format of kafka which collected from mongodb via debezium #4870
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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
34 changes: 34 additions & 0 deletions
34
.../main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumBsonDataFormat.java
This file contains hidden or 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,34 @@ | ||
| /* | ||
| * 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.format.AbstractJsonDataFormat; | ||
| import org.apache.paimon.flink.action.cdc.format.RecordParserFactory; | ||
|
|
||
| /** | ||
| * Supports the message queue's debezium bson data format and provides definitions for the message | ||
| * queue's record json deserialization class and parsing class {@link DebeziumBsonRecordParser}. | ||
| */ | ||
| public class DebeziumBsonDataFormat extends AbstractJsonDataFormat { | ||
|
|
||
| @Override | ||
| protected RecordParserFactory parser() { | ||
| return DebeziumBsonRecordParser::new; | ||
| } | ||
| } |
38 changes: 38 additions & 0 deletions
38
...ava/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumBsonDataFormatFactory.java
This file contains hidden or 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,38 @@ | ||
| /* | ||
| * 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.format.DataFormat; | ||
| import org.apache.paimon.flink.action.cdc.format.DataFormatFactory; | ||
|
|
||
| /** Factory to create {@link DebeziumBsonDataFormat}. */ | ||
| public class DebeziumBsonDataFormatFactory implements DataFormatFactory { | ||
|
|
||
| public static final String IDENTIFIER = "debezium-bson"; | ||
|
|
||
| @Override | ||
| public String identifier() { | ||
| return IDENTIFIER; | ||
| } | ||
|
|
||
| @Override | ||
| public DataFormat create() { | ||
| return new DebeziumBsonDataFormat(); | ||
| } | ||
| } |
131 changes: 131 additions & 0 deletions
131
...ain/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumBsonRecordParser.java
This file contains hidden or 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,131 @@ | ||
| /* | ||
| * 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.CdcSourceRecord; | ||
| import org.apache.paimon.flink.action.cdc.ComputedColumn; | ||
| import org.apache.paimon.flink.action.cdc.TypeMapping; | ||
| import org.apache.paimon.flink.action.cdc.mongodb.BsonValueConvertor; | ||
| import org.apache.paimon.types.DataTypes; | ||
| import org.apache.paimon.types.RowType; | ||
| import org.apache.paimon.utils.Preconditions; | ||
| import org.apache.paimon.utils.TypeUtils; | ||
|
|
||
| import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; | ||
| import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; | ||
|
|
||
| import org.bson.BsonDocument; | ||
| import org.bson.BsonValue; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import javax.annotation.Nullable; | ||
|
|
||
| import java.util.Collections; | ||
| import java.util.LinkedHashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Objects; | ||
|
|
||
| import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_PAYLOAD; | ||
| import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.FIELD_SCHEMA; | ||
| import static org.apache.paimon.utils.JsonSerdeUtil.writeValueAsString; | ||
|
|
||
| /** | ||
| * The {@code DebeziumRecordParser} class extends the {@link DebeziumJsonRecordParser} and is | ||
| * designed to parse records from MongoDB's BSON change data capture (CDC) format via Debezium. | ||
| * | ||
| * <p>The class supports The following features: | ||
| * | ||
| * <p>Convert bson string to java object from before/after field | ||
| * | ||
| * <p>Parse the schema from bson, all fields are string type, and the _id field is the primary key | ||
| */ | ||
| public class DebeziumBsonRecordParser extends DebeziumJsonRecordParser { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(DebeziumBsonRecordParser.class); | ||
|
|
||
| private static final String FIELD_COLLECTION = "collection"; | ||
| private static final String FIELD_OBJECT_ID = "_id"; | ||
| private static final List<String> PRIMARY_KEYS = Collections.singletonList(FIELD_OBJECT_ID); | ||
|
|
||
| public DebeziumBsonRecordParser(TypeMapping typeMapping, List<ComputedColumn> computedColumns) { | ||
| super(typeMapping, computedColumns); | ||
| } | ||
|
|
||
| @Override | ||
| protected void setRoot(CdcSourceRecord record) { | ||
| JsonNode node = (JsonNode) record.getValue(); | ||
| if (node.has(FIELD_SCHEMA)) { | ||
| root = node.get(FIELD_PAYLOAD); | ||
| } else { | ||
| root = node; | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| protected Map<String, String> extractRowData(JsonNode record, RowType.Builder rowTypeBuilder) { | ||
| // bson record should be a string | ||
| Preconditions.checkArgument( | ||
| record.isTextual(), | ||
| "debezium bson record expected to be STRING type, but actual is %s", | ||
| record.getNodeType()); | ||
|
|
||
| BsonDocument document = BsonDocument.parse(record.asText()); | ||
| LinkedHashMap<String, String> resultMap = new LinkedHashMap<>(); | ||
| for (Map.Entry<String, BsonValue> entry : document.entrySet()) { | ||
| String fieldName = entry.getKey(); | ||
| resultMap.put(fieldName, toJsonString(BsonValueConvertor.convert(entry.getValue()))); | ||
| rowTypeBuilder.field(fieldName, DataTypes.STRING()); | ||
| } | ||
|
|
||
| evalComputedColumns(resultMap, rowTypeBuilder); | ||
|
|
||
| return resultMap; | ||
| } | ||
|
|
||
| private static String toJsonString(Object entry) { | ||
| if (entry == null) { | ||
| return null; | ||
| } else if (!TypeUtils.isBasicType(entry)) { | ||
| try { | ||
| return writeValueAsString(entry); | ||
| } catch (JsonProcessingException e) { | ||
| LOG.error("Failed to deserialize record.", e); | ||
| } | ||
| } | ||
| return Objects.toString(entry); | ||
| } | ||
|
|
||
| @Override | ||
| protected List<String> extractPrimaryKeys() { | ||
| return PRIMARY_KEYS; | ||
| } | ||
|
|
||
| @Nullable | ||
| @Override | ||
| protected String getTableName() { | ||
| return getFromSourceField(FIELD_COLLECTION); | ||
| } | ||
|
|
||
| @Override | ||
| protected String format() { | ||
| return "debezium-bson"; | ||
| } | ||
| } | ||
This file contains hidden or 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.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we can map bson type to actual paimon type.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Because mongodb does not have a schema, the data type of the same field in different documents may be different. For safety reasons, string type is used. By the way, this is the same as mongodb-cdc
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In addition, BsonValueConvertor is used here for data conversion, while mongodb-cdc does not convert data. There will be inconsistencies here, and it may be necessary to discuss.
For example:
The data of mongodb-cdc is:
{ "_id": "{\"$oid\":\"64001c996f4de7ff3189d374\"}", "updated_at": "{\"$numberLong\":\"1732232838425\"}}" }The data after BsonValueConvertor conversion is:
{ "_id": "64001c996f4de7ff3189d374", "updated_at": "1732232838425" }I think it is also possible to configure whether to use BsonValueConvertor for conversion through TypeMapping