forked from apache/paimon
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Kafka debezium json supports automatic discovery of primary keys
- Loading branch information
1 parent
3baca1c
commit b1c1a2f
Showing
13 changed files
with
359 additions
and
84 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
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
94 changes: 94 additions & 0 deletions
94
...ain/java/org/apache/paimon/flink/action/cdc/kafka/KafkaKeyValueDeserializationSchema.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,94 @@ | ||
/* | ||
* 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.CdcSourceRecord; | ||
|
||
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; | ||
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; | ||
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; | ||
|
||
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.kafka.clients.consumer.ConsumerRecord; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.io.IOException; | ||
|
||
import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; | ||
|
||
/** A kafka key value deserialization schema for {@link CdcSourceRecord}. */ | ||
public class KafkaKeyValueDeserializationSchema | ||
implements KafkaRecordDeserializationSchema<CdcSourceRecord> { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private static final Logger LOG = | ||
LoggerFactory.getLogger(KafkaKeyValueDeserializationSchema.class); | ||
|
||
private final ObjectMapper objectMapper = new ObjectMapper(); | ||
|
||
public KafkaKeyValueDeserializationSchema() { | ||
objectMapper | ||
.configure(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS, true) | ||
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); | ||
} | ||
|
||
public CdcSourceRecord deserialize(ConsumerRecord<byte[], byte[]> record) throws IOException { | ||
if (record == null || record.value() == null) { | ||
return null; | ||
} | ||
try { | ||
JsonNode key = | ||
record.key() != null | ||
? objectMapper.readValue(record.key(), JsonNode.class) | ||
: null; | ||
CdcSourceRecord sourceRecord = | ||
new CdcSourceRecord( | ||
key, objectMapper.readValue(record.value(), JsonNode.class)); | ||
return sourceRecord; | ||
} catch (Exception e) { | ||
LOG.error("Invalid Json:\n{}", new String(record.value())); | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public void deserialize( | ||
ConsumerRecord<byte[], byte[]> record, Collector<CdcSourceRecord> collector) | ||
throws IOException { | ||
try { | ||
CdcSourceRecord sourceRecord = deserialize(record); | ||
if (sourceRecord == null) { | ||
return; | ||
} | ||
collector.collect(sourceRecord); | ||
} catch (Exception e) { | ||
LOG.error("Invalid Json:\n{}", new String(record.value())); | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public TypeInformation<CdcSourceRecord> getProducedType() { | ||
return getForClass(CdcSourceRecord.class); | ||
} | ||
} |
71 changes: 0 additions & 71 deletions
71
.../org/apache/paimon/flink/action/cdc/kafka/KafkaValueOnlyDeserializationSchemaWrapper.java
This file was deleted.
Oops, something went wrong.
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.