Skip to content

Commit

Permalink
Kafka debezium json supports automatic discovery of primary keys
Browse files Browse the repository at this point in the history
  • Loading branch information
sunxiaojian committed Mar 27, 2024
1 parent f70f9d0 commit 138732c
Show file tree
Hide file tree
Showing 14 changed files with 399 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.SerializerProvider;
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer;
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.module.SimpleModule;
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer;

import javax.annotation.Nullable;
Expand Down Expand Up @@ -128,6 +129,13 @@ public static <T> T fromJson(String json, TypeReference<T> typeReference) {
}
}

public static <T> ObjectNode setNode(ObjectNode node, String fieldName, T value) {
ObjectMapper mapper = new ObjectMapper();
JsonNode nodeValue = OBJECT_MAPPER_INSTANCE.valueToTree(value);
node.set(fieldName, nodeValue);
return node;
}

public static <T> T fromJson(String json, Class<T> clazz) {
try {
return OBJECT_MAPPER_INSTANCE.reader().readValue(json, clazz);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,9 @@
import org.apache.paimon.flink.action.cdc.SyncTableActionBase.SchemaRetrievalException;
import org.apache.paimon.flink.action.cdc.format.DataFormat;
import org.apache.paimon.flink.action.cdc.format.RecordParser;
import org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.utils.Pair;

import java.util.Collections;
import java.util.List;
Expand Down Expand Up @@ -55,6 +57,7 @@ public static Schema getSchema(
while (true) {
Optional<Schema> schema =
consumer.getRecords(POLL_TIMEOUT_MILLIS).stream()
.map(DebeziumSchemaUtils::extractPrimaryKeys)
.map(recordParser::buildSchema)
.filter(Objects::nonNull)
.findFirst();
Expand Down Expand Up @@ -89,7 +92,7 @@ private static void sleepSafely(int duration) {
/** Wrap the consumer for different message queues. */
public interface ConsumerWrapper extends AutoCloseable {

List<String> getRecords(int pollTimeOutMills);
List<Pair<String, String>> getRecords(int pollTimeOutMills);

String topic();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,7 @@ protected void evalComputedColumns(
});
}

private List<String> extractPrimaryKeys() {
protected List<String> extractPrimaryKeys() {
ArrayNode pkNames = getNodeAs(root, primaryField(), ArrayNode.class);
if (pkNames == null) {
return Collections.emptyList();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,12 +64,12 @@
*/
public class DebeziumRecordParser extends RecordParser {

private static final String FIELD_SCHEMA = "schema";
protected static final String FIELD_PAYLOAD = "payload";
public static final String FIELD_SCHEMA = "schema";
public static final String FIELD_PAYLOAD = "payload";
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";
public 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";
Expand All @@ -82,6 +82,8 @@ public class DebeziumRecordParser extends RecordParser {
private final Map<String, String> classNames = new HashMap<>();
private final Map<String, Map<String, String>> parameters = new HashMap<>();

private final List<String> primaryKeys = new ArrayList<>();

public DebeziumRecordParser(
boolean caseSensitive, TypeMapping typeMapping, List<ComputedColumn> computedColumns) {
super(caseSensitive, typeMapping, computedColumns);
Expand Down Expand Up @@ -121,7 +123,7 @@ private JsonNode getBefore(String op) {
@Override
protected void setRoot(String record) {
JsonNode node = JsonSerdeUtil.fromJson(record, JsonNode.class);

preparePrimaryKeysIfNeed(node);
hasSchema = false;
if (node.has(FIELD_SCHEMA)) {
root = node.get(FIELD_PAYLOAD);
Expand All @@ -135,6 +137,16 @@ protected void setRoot(String record) {
}
}

private void preparePrimaryKeysIfNeed(JsonNode node) {
primaryKeys.clear();
if (node.has(FIELD_PRIMARY)) {
ArrayNode primaryKeyNode = (ArrayNode) node.get(FIELD_PRIMARY);
for (JsonNode element : primaryKeyNode) {
primaryKeys.add(element.asText());
}
}
}

private void parseSchema(JsonNode schema) {
debeziumTypes.clear();
classNames.clear();
Expand Down Expand Up @@ -218,6 +230,11 @@ protected Map<String, String> extractRowData(
return resultMap;
}

@Override
protected List<String> extractPrimaryKeys() {
return primaryKeys;
}

@Override
protected String primaryField() {
return FIELD_PRIMARY;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,14 @@
import org.apache.paimon.types.DataTypes;
import org.apache.paimon.types.DecimalType;
import org.apache.paimon.utils.DateTimeUtils;
import org.apache.paimon.utils.JsonSerdeUtil;
import org.apache.paimon.utils.Pair;
import org.apache.paimon.utils.StringUtils;

import org.apache.paimon.shade.guava30.com.google.common.collect.Lists;
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode;
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;

import io.debezium.data.Bits;
import io.debezium.data.geometry.Geometry;
Expand All @@ -46,16 +51,48 @@
import java.time.ZoneId;
import java.time.ZoneOffset;
import java.util.Base64;
import java.util.List;
import java.util.Map;

import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.TO_STRING;
import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumRecordParser.FIELD_PAYLOAD;
import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumRecordParser.FIELD_PRIMARY;
import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumRecordParser.FIELD_SCHEMA;

/**
* Utils to handle 'schema' field in debezium Json. TODO: The methods have many duplicate codes with
* MySqlRecordParser. Need refactor.
*/
public class DebeziumSchemaUtils {

/** Append primary keys to value. */
public static String extractPrimaryKeys(Pair<String, String> record) {
String key = record.getKey();
if (StringUtils.isBlank(key)) {
return record.getValue();
}
try {
List<String> primaryKeys = Lists.newArrayList();
JsonNode keyNode = JsonSerdeUtil.fromJson(key, JsonNode.class);
JsonNode payload;
if (keyNode.has(FIELD_SCHEMA)) {
payload = keyNode.get(FIELD_PAYLOAD);
} else {
payload = keyNode;
}
payload.fieldNames().forEachRemaining(primaryKeys::add);
ObjectNode valueNode = JsonSerdeUtil.fromJson(record.getValue(), ObjectNode.class);

// append primary keys
JsonSerdeUtil.setNode(valueNode, FIELD_PRIMARY, primaryKeys);
return JsonSerdeUtil.writeValueAsString(valueNode);
} catch (JsonProcessingException e) {
throw new RuntimeException(
"An error occurred when automatically attaching the debezium primary keys to Value",
e);
}
}

/** Transform raw string value according to schema. */
public static String transformRawValue(
@Nullable String rawValue,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,9 @@

import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils;
import org.apache.paimon.flink.action.cdc.format.DataFormat;
import org.apache.paimon.utils.Pair;
import org.apache.paimon.utils.StringUtils;

import org.apache.flink.api.common.serialization.SimpleStringSchema;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.kafka.source.KafkaSource;
import org.apache.flink.connector.kafka.source.KafkaSourceBuilder;
Expand All @@ -34,7 +34,6 @@
import org.apache.flink.table.api.ValidationException;
import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
Expand Down Expand Up @@ -84,7 +83,10 @@ public static KafkaSource<String> buildKafkaSource(Configuration kafkaConfig) {
}

kafkaSourceBuilder
.setValueOnlyDeserializer(new SimpleStringSchema())
// .setValueOnlyDeserializer(new SimpleStringSchema())
.setDeserializer(
new KafkaKeyValueDeserializationSchema(
kafkaConfig.get(KafkaConnectorOptions.VALUE_FORMAT)))
.setGroupId(kafkaPropertiesGroupId(kafkaConfig));
Properties properties = createKafkaProperties(kafkaConfig);

Expand Down Expand Up @@ -316,11 +318,11 @@ private static class KafkaConsumerWrapper implements MessageQueueSchemaUtils.Con
}

@Override
public List<String> getRecords(int pollTimeOutMills) {
public List<Pair<String, String>> getRecords(int pollTimeOutMills) {
ConsumerRecords<String, String> consumerRecords =
consumer.poll(Duration.ofMillis(pollTimeOutMills));
return StreamSupport.stream(consumerRecords.records(topic).spliterator(), false)
.map(ConsumerRecord::value)
.map(record -> Pair.of(record.key(), record.value()))
.collect(Collectors.toList());
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* 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.flink.action.cdc.format.DataFormat;
import org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils;
import org.apache.paimon.utils.Pair;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
import org.apache.flink.util.Collector;
import org.apache.flink.util.Preconditions;
import org.apache.kafka.clients.consumer.ConsumerRecord;

import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.Objects;

/** Deserialization for kafka key and value. */
public class KafkaKeyValueDeserializationSchema
implements KafkaRecordDeserializationSchema<String> {

private String charset;
private String format;

public KafkaKeyValueDeserializationSchema(String format) {
this(StandardCharsets.UTF_8.name(), format);
}

public KafkaKeyValueDeserializationSchema(String charset, String format) {
this.charset = Preconditions.checkNotNull(charset);
this.format = format;
}

@Override
public void deserialize(ConsumerRecord<byte[], byte[]> record, Collector<String> collector)
throws IOException {
DataFormat dataFormat = DataFormat.fromConfigString(format);
String value = new String(record.value(), Charset.forName(charset));
switch (dataFormat) {
case DEBEZIUM_JSON:
if (Objects.isNull(record.key())) {
collector.collect(value);
} else {
String key = new String(record.key(), Charset.forName(charset));
Pair<String, String> keyValue = Pair.of(key, value);
String result = DebeziumSchemaUtils.extractPrimaryKeys(keyValue);
collector.collect(result);
}
break;
default:
collector.collect(value);
break;
}
}

@Override
public TypeInformation<String> getProducedType() {
return TypeInformation.of(String.class);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.paimon.flink.action.cdc.MessageQueueSchemaUtils;
import org.apache.paimon.flink.action.cdc.format.DataFormat;
import org.apache.paimon.utils.Pair;

import org.apache.flink.api.common.serialization.SimpleStringSchema;
import org.apache.flink.configuration.ConfigOption;
Expand All @@ -34,6 +35,7 @@
import org.apache.flink.connector.pulsar.source.enumerator.subscriber.impl.TopicPatternSubscriber;
import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
import org.apache.flink.connector.pulsar.source.reader.PulsarPartitionSplitReader;
import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.ConsumerBuilder;
import org.apache.pulsar.client.api.KeySharedPolicy;
Expand Down Expand Up @@ -170,6 +172,9 @@ public static PulsarSource<String> buildPulsarSource(Configuration pulsarConfig)
.setServiceUrl(pulsarConfig.get(PULSAR_SERVICE_URL))
.setAdminUrl(pulsarConfig.get(PULSAR_ADMIN_URL))
.setSubscriptionName(pulsarConfig.get(PULSAR_SUBSCRIPTION_NAME))
.setDeserializationSchema(
new PulsarKeyValueDeserializationSchema(
pulsarConfig.get(KafkaConnectorOptions.VALUE_FORMAT)))
.setDeserializationSchema(new SimpleStringSchema());

pulsarConfig.getOptional(TOPIC).ifPresent(pulsarSourceBuilder::setTopics);
Expand Down Expand Up @@ -382,12 +387,12 @@ private static class PulsarConsumerWrapper implements MessageQueueSchemaUtils.Co
}

@Override
public List<String> getRecords(int pollTimeOutMills) {
public List<Pair<String, String>> getRecords(int pollTimeOutMills) {
try {
Message<String> message = consumer.receive(pollTimeOutMills, TimeUnit.MILLISECONDS);
return message == null
? Collections.emptyList()
: Collections.singletonList(message.getValue());
: Collections.singletonList(Pair.of(message.getKey(), message.getValue()));
} catch (PulsarClientException e) {
throw new RuntimeException(e);
}
Expand Down
Loading

0 comments on commit 138732c

Please sign in to comment.