From 350577d0ae0877352c880f600d5dbb007d65889a Mon Sep 17 00:00:00 2001 From: jimmy <961370183@qq.com> Date: Sat, 6 Sep 2025 23:51:59 +0800 Subject: [PATCH] MINOR: Add doc for external schemas in JSONConverter (#20429) This is a follow-up to #19449, which do the following things: 1. Add document to explain `schema.content` only work for sink connector when `schemas.enable` set to true. 2. Handle the case that while jsonValue contains the `schema` and `payload` fields, we should use the corresponding value. Reviewers: Priyanka K U , Chia-Ping Tsai --- .../main/java/org/apache/kafka/connect/json/JsonConverter.java | 2 +- .../org/apache/kafka/connect/json/JsonConverterConfig.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index 6c999e3e6a0..dac2ce56741 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -222,7 +222,7 @@ public class JsonConverter implements Converter, HeaderConverter, Versioned { private JsonConverterConfig config; private Cache fromConnectSchemaCache; private Cache toConnectSchemaCache; - private Schema schema = null; // if a schema is provided in config, this schema will be used for all messages + private Schema schema = null; // if a schema is provided in config, this schema will be used for all messages for sink connector private final JsonSerializer serializer; private final JsonDeserializer deserializer; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java index 4d148250114..17d48c7f14c 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java @@ -38,7 +38,8 @@ public final class JsonConverterConfig extends ConverterConfig { public static final String SCHEMA_CONTENT_CONFIG = "schema.content"; public static final String SCHEMA_CONTENT_DEFAULT = null; - private static final String SCHEMA_CONTENT_DOC = "When set, this is used as the schema for all messages. Otherwise, the schema will be included in the content of each message."; + private static final String SCHEMA_CONTENT_DOC = "When set, this is used as the schema for all messages, and the schemas within each of the message will be ignored." + + "Otherwise, the schema will be included in the content of each message. This configuration applies only 'schemas.enable' is true, and it exclusively affects the sink connector."; private static final String SCHEMA_CONTENT_DISPLAY = "Schema Content"; public static final String SCHEMAS_CACHE_SIZE_CONFIG = "schemas.cache.size";