-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[cdc] Use CdcSourceRecord as source type instead of raw String
- Loading branch information
Showing
27 changed files
with
399 additions
and
116 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
95 changes: 95 additions & 0 deletions
95
...dc/src/main/java/org/apache/paimon/flink/action/cdc/CdcDebeziumDeserializationSchema.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,95 @@ | ||
/* | ||
* 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; | ||
|
||
import com.ververica.cdc.debezium.DebeziumDeserializationSchema; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
import org.apache.flink.util.Collector; | ||
import org.apache.kafka.connect.json.JsonConverter; | ||
import org.apache.kafka.connect.json.JsonConverterConfig; | ||
import org.apache.kafka.connect.source.SourceRecord; | ||
import org.apache.kafka.connect.storage.ConverterConfig; | ||
import org.apache.kafka.connect.storage.ConverterType; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
|
||
import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; | ||
|
||
/** | ||
* A JSON format implementation of {@link DebeziumDeserializationSchema} which deserializes the | ||
* received {@link SourceRecord} to {@link CdcSourceRecord}. | ||
*/ | ||
public class CdcDebeziumDeserializationSchema | ||
implements DebeziumDeserializationSchema<CdcSourceRecord> { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private transient JsonConverter jsonConverter; | ||
|
||
/** | ||
* Configuration whether to enable {@link JsonConverterConfig#SCHEMAS_ENABLE_CONFIG} to include | ||
* schema in messages. | ||
*/ | ||
private final Boolean includeSchema; | ||
|
||
/** The custom configurations for {@link JsonConverter}. */ | ||
private final Map<String, Object> customConverterConfigs; | ||
|
||
public CdcDebeziumDeserializationSchema() { | ||
this(false); | ||
} | ||
|
||
public CdcDebeziumDeserializationSchema(Boolean includeSchema) { | ||
this(includeSchema, null); | ||
} | ||
|
||
public CdcDebeziumDeserializationSchema( | ||
Boolean includeSchema, Map<String, Object> customConverterConfigs) { | ||
this.includeSchema = includeSchema; | ||
this.customConverterConfigs = customConverterConfigs; | ||
} | ||
|
||
@Override | ||
public void deserialize(SourceRecord record, Collector<CdcSourceRecord> out) throws Exception { | ||
if (jsonConverter == null) { | ||
initializeJsonConverter(); | ||
} | ||
byte[] bytes = | ||
jsonConverter.fromConnectData(record.topic(), record.valueSchema(), record.value()); | ||
out.collect(new CdcSourceRecord(record.topic(), null, new String(bytes))); | ||
} | ||
|
||
/** Initialize {@link JsonConverter} with given configs. */ | ||
private void initializeJsonConverter() { | ||
jsonConverter = new JsonConverter(); | ||
final HashMap<String, Object> configs = new HashMap<>(2); | ||
configs.put(ConverterConfig.TYPE_CONFIG, ConverterType.VALUE.getName()); | ||
configs.put(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, includeSchema); | ||
if (customConverterConfigs != null) { | ||
configs.putAll(customConverterConfigs); | ||
} | ||
jsonConverter.configure(configs); | ||
} | ||
|
||
@Override | ||
public TypeInformation<CdcSourceRecord> getProducedType() { | ||
return getForClass(CdcSourceRecord.class); | ||
} | ||
} |
62 changes: 62 additions & 0 deletions
62
...-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcDeserializationSchema.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,62 @@ | ||
/* | ||
* 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; | ||
|
||
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.serialization.DeserializationSchema; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
|
||
import java.io.IOException; | ||
|
||
import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; | ||
|
||
/** A simple deserialization schema for {@link CdcSourceRecord}. */ | ||
public class CdcDeserializationSchema implements DeserializationSchema<CdcSourceRecord> { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private final ObjectMapper objectMapper = new ObjectMapper(); | ||
|
||
public CdcDeserializationSchema() { | ||
objectMapper | ||
.configure(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS, true) | ||
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); | ||
} | ||
|
||
@Override | ||
public CdcSourceRecord deserialize(byte[] message) throws IOException { | ||
if (message == null) { | ||
return null; | ||
} | ||
return new CdcSourceRecord(objectMapper.readValue(message, JsonNode.class)); | ||
} | ||
|
||
@Override | ||
public boolean isEndOfStream(CdcSourceRecord nextElement) { | ||
return false; | ||
} | ||
|
||
@Override | ||
public TypeInformation<CdcSourceRecord> getProducedType() { | ||
return getForClass(CdcSourceRecord.class); | ||
} | ||
} |
85 changes: 85 additions & 0 deletions
85
...nk/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcSourceRecord.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,85 @@ | ||
/* | ||
* 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; | ||
|
||
import org.apache.paimon.annotation.Experimental; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
import java.io.Serializable; | ||
import java.util.Objects; | ||
|
||
/** A data change record from the CDC source. */ | ||
@Experimental | ||
public class CdcSourceRecord implements Serializable { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
@Nullable private final String topic; | ||
|
||
@Nullable private final Object key; | ||
|
||
private final Object value; | ||
|
||
public CdcSourceRecord(@Nullable String topic, @Nullable Object key, Object value) { | ||
this.topic = topic; | ||
this.key = key; | ||
this.value = value; | ||
} | ||
|
||
public CdcSourceRecord(Object value) { | ||
this(null, null, value); | ||
} | ||
|
||
@Nullable | ||
public String getTopic() { | ||
return topic; | ||
} | ||
|
||
@Nullable | ||
public Object getKey() { | ||
return key; | ||
} | ||
|
||
public Object getValue() { | ||
return value; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (!(o instanceof CdcSourceRecord)) { | ||
return false; | ||
} | ||
|
||
CdcSourceRecord that = (CdcSourceRecord) o; | ||
return Objects.equals(topic, that.topic) | ||
&& Objects.equals(key, that.key) | ||
&& Objects.equals(value, that.value); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(topic, key, value); | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return topic + ": " + key + " " + value; | ||
} | ||
} |
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
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.