diff --git a/seatunnel-connectors-v2/connector-kafka/pom.xml b/seatunnel-connectors-v2/connector-kafka/pom.xml
index 75e56237b24..4159e8783aa 100644
--- a/seatunnel-connectors-v2/connector-kafka/pom.xml
+++ b/seatunnel-connectors-v2/connector-kafka/pom.xml
@@ -41,6 +41,11 @@
kafka-clients
${kafka.client.version}
+
+ org.apache.seatunnel
+ seatunnel-format-json
+ ${project.version}
+
\ No newline at end of file
diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java
index ee1492044e6..d48d12cf646 100644
--- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java
+++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java
@@ -31,7 +31,7 @@ public class Config {
/**
* The server address of kafka cluster.
*/
- public static final String BOOTSTRAP_SERVER = "bootstrap.server";
+ public static final String BOOTSTRAP_SERVERS = "bootstrap.servers";
public static final String KAFKA_CONFIG_PREFIX = "kafka.";
diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java
index ad9a24ef27a..e80a7d306b0 100644
--- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java
+++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java
@@ -19,32 +19,22 @@
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
-import org.apache.seatunnel.common.config.Common;
-import org.apache.seatunnel.common.utils.JsonUtils;
+import org.apache.seatunnel.format.json.JsonSerializationSchema;
import org.apache.kafka.clients.producer.ProducerRecord;
-import java.util.HashMap;
-import java.util.Map;
-
-public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer {
+public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer {
private final String topic;
- private final SeaTunnelRowType seaTunnelRowType;
+ private final JsonSerializationSchema jsonSerializationSchema;
public DefaultSeaTunnelRowSerializer(String topic, SeaTunnelRowType seaTunnelRowType) {
this.topic = topic;
- this.seaTunnelRowType = seaTunnelRowType;
+ this.jsonSerializationSchema = new JsonSerializationSchema(seaTunnelRowType);
}
@Override
- public ProducerRecord serializeRow(SeaTunnelRow row) {
- Map