DBZ-8040: Custom Converter Support

This commit is contained in:
Ryan van Huuksloot 2024-07-11 13:25:23 -04:00 committed by Chris Cranford
parent ef7a9747db
commit 479a41af28
2 changed files with 25 additions and 1 deletions

View File

@ -0,0 +1,11 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.embedded;
import io.debezium.engine.format.SerializationFormat;
public class ClientProvided implements SerializationFormat<Object> {
}

View File

@ -171,6 +171,12 @@ private HeaderConverter createHeaderConverter(Class<? extends SerializationForma
if (isFormat(format, Json.class) || isFormat(format, JsonByteArray.class)) {
converterConfig = converterConfig.edit().withDefault(FIELD_CLASS, "org.apache.kafka.connect.json.JsonConverter").build();
}
else if (isFormat(format, ClientProvided.class)) {
if (converterConfig.getString(FIELD_CLASS) == null) {
throw new DebeziumException(
"`" + ClientProvided.class.getSimpleName().toLowerCase() + "`" + " header converter requires a '" + FIELD_CLASS + "' configuration");
}
}
else {
throw new DebeziumException("Header Converter '" + format.getSimpleName() + "' is not supported");
}
@ -218,6 +224,13 @@ else if (isFormat(format, Binary.class)) {
else if (isFormat(format, SimpleString.class)) {
converterConfig = converterConfig.edit().withDefault(FIELD_CLASS, "org.apache.kafka.connect.storage.StringConverter").build();
}
else if (isFormat(format, ClientProvided.class)) {
if (converterConfig.getString(FIELD_CLASS) == null) {
throw new DebeziumException(
"`" + ClientProvided.class.getSimpleName().toLowerCase() + "`" + (key ? " key" : " value") + " converter requires a '" + FIELD_CLASS
+ "' configuration");
}
}
else {
throw new DebeziumException("Converter '" + format.getSimpleName() + "' is not supported");
}