-
Notifications
You must be signed in to change notification settings - Fork 2k
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
[FLINK-36611][pipeline-connector][kafka] Add schema info to output of Kafka sink #3791
Draft
MOBIN-F
wants to merge
7
commits into
apache:master
Choose a base branch
from
MOBIN-F:release-support-debezium-json-include-schema
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Draft
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
b712e2f
Add schema fields to DataChangeEvent.
MOBIN-F 3bd3e47
get schema json through jsonConverter.asJsonSchema
MOBIN-F b874ab4
support pipeline-connector-mysql and pipeline-connector-kafka output …
MOBIN-F c18bf0d
support pipeline-connector-values output schema info
MOBIN-F 36724f3
add e2eTest
MOBIN-F c72d25c
fix code style
MOBIN-F 61ba472
Merge remote-tracking branch 'origin/master' into release-support-deb…
MOBIN-F 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 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
148 changes: 148 additions & 0 deletions
148
...ache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonRowDataSerializationSchema.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,148 @@ | ||
/* | ||
* 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.connectors.kafka.json.debezium; | ||
|
||
import org.apache.flink.api.common.serialization.SerializationSchema; | ||
import org.apache.flink.formats.common.TimestampFormat; | ||
import org.apache.flink.formats.json.JsonFormatOptions; | ||
import org.apache.flink.formats.json.JsonParserRowDataDeserializationSchema; | ||
import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; | ||
import org.apache.flink.formats.json.RowDataToJsonConverters; | ||
import org.apache.flink.table.data.RowData; | ||
import org.apache.flink.table.types.logical.RowType; | ||
import org.apache.flink.util.jackson.JacksonMapperFactory; | ||
|
||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; | ||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; | ||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; | ||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; | ||
|
||
import java.util.Objects; | ||
|
||
/** | ||
* Serialization schema that serializes an object of Flink internal data structure into a JSON | ||
* bytes. | ||
* | ||
* <p>Serializes the input Flink object into a JSON string and converts it into <code>byte[]</code>. | ||
* | ||
* <p>Result <code>byte[]</code> messages can be deserialized using {@link | ||
* JsonRowDataDeserializationSchema} or {@link JsonParserRowDataDeserializationSchema}. | ||
*/ | ||
public class DebeziumJsonRowDataSerializationSchema implements SerializationSchema<RowData> { | ||
private static final long serialVersionUID = 1L; | ||
|
||
/** RowType to generate the runtime converter. */ | ||
private final RowType rowType; | ||
|
||
/** The converter that converts internal data formats to JsonNode. */ | ||
private final RowDataToJsonConverters.RowDataToJsonConverter runtimeConverter; | ||
|
||
/** Object mapper that is used to create output JSON objects. */ | ||
private transient ObjectMapper mapper; | ||
|
||
/** Reusable object node. */ | ||
private transient ObjectNode node; | ||
|
||
/** Timestamp format specification which is used to parse timestamp. */ | ||
private final TimestampFormat timestampFormat; | ||
|
||
/** The handling mode when serializing null keys for map data. */ | ||
private final JsonFormatOptions.MapNullKeyMode mapNullKeyMode; | ||
|
||
/** The string literal when handling mode for map null key LITERAL. */ | ||
private final String mapNullKeyLiteral; | ||
|
||
/** Flag indicating whether to serialize all decimals as plain numbers. */ | ||
private final boolean encodeDecimalAsPlainNumber; | ||
|
||
private final boolean includeSchemaInfo; | ||
|
||
public DebeziumJsonRowDataSerializationSchema( | ||
RowType rowType, | ||
TimestampFormat timestampFormat, | ||
JsonFormatOptions.MapNullKeyMode mapNullKeyMode, | ||
String mapNullKeyLiteral, | ||
boolean encodeDecimalAsPlainNumber, | ||
boolean includeSchemaInfo) { | ||
this.rowType = rowType; | ||
this.timestampFormat = timestampFormat; | ||
this.mapNullKeyMode = mapNullKeyMode; | ||
this.mapNullKeyLiteral = mapNullKeyLiteral; | ||
this.encodeDecimalAsPlainNumber = encodeDecimalAsPlainNumber; | ||
this.runtimeConverter = | ||
new RowDataToJsonConverters(timestampFormat, mapNullKeyMode, mapNullKeyLiteral) | ||
.createConverter(rowType); | ||
this.includeSchemaInfo = includeSchemaInfo; | ||
} | ||
|
||
@Override | ||
public void open(InitializationContext context) throws Exception { | ||
mapper = | ||
JacksonMapperFactory.createObjectMapper() | ||
.configure( | ||
JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, | ||
encodeDecimalAsPlainNumber); | ||
} | ||
|
||
@Override | ||
public byte[] serialize(RowData row) { | ||
if (node == null) { | ||
node = mapper.createObjectNode(); | ||
} | ||
|
||
try { | ||
runtimeConverter.convert(mapper, node, row); | ||
if (includeSchemaInfo) { | ||
// schema is a nested json string, asText() can return a pure string without other | ||
// escape characters such as "\" | ||
String schemaValue = node.get("schema").asText(); | ||
JsonNode schemaNode = mapper.readTree(schemaValue); | ||
node.set("schema", schemaNode); | ||
} | ||
return mapper.writeValueAsBytes(node); | ||
} catch (Throwable t) { | ||
throw new RuntimeException(String.format("Could not serialize row '%s'.", row), t); | ||
} | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
DebeziumJsonRowDataSerializationSchema that = (DebeziumJsonRowDataSerializationSchema) o; | ||
return rowType.equals(that.rowType) | ||
&& timestampFormat.equals(that.timestampFormat) | ||
&& mapNullKeyMode.equals(that.mapNullKeyMode) | ||
&& mapNullKeyLiteral.equals(that.mapNullKeyLiteral) | ||
&& encodeDecimalAsPlainNumber == that.encodeDecimalAsPlainNumber; | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash( | ||
rowType, | ||
timestampFormat, | ||
mapNullKeyMode, | ||
mapNullKeyLiteral, | ||
encodeDecimalAsPlainNumber); | ||
} | ||
} |
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.
Because the schema is passed to the downstream as a string, and there is a nested json in the schema, if the json string is put into jsonNode, there will be ["]. The JsonNode.asText() method can solve this problem well.