Skip to content

Commit a7975c0

Browse files
Priyanka K UPriyanka K U
authored andcommitted
feat: addressing review comments
Contributes to: event-integration/eventstreams-planning/12766 KIP Link: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1054%3A+Support+external+schemas+in+JSONConverte Signed-off-by: Priyanka K U <[email protected]>
1 parent 5cd489f commit a7975c0

File tree

1 file changed

+22
-5
lines changed

1 file changed

+22
-5
lines changed

connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java

Lines changed: 22 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,8 @@
3636

3737
import org.junit.jupiter.api.BeforeEach;
3838
import org.junit.jupiter.api.Test;
39+
import org.junit.jupiter.params.ParameterizedTest;
40+
import org.junit.jupiter.params.provider.ValueSource;
3941

4042
import java.io.File;
4143
import java.io.IOException;
@@ -980,37 +982,52 @@ public void testVersionRetrievedFromAppInfoParser() {
980982

981983
@Test
982984
public void testSchemaContentIsNull() {
983-
converter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, null), false);
985+
converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, null), false);
984986
assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes()));
985987
}
986988

987989
@Test
988990
public void testSchemaContentIsEmptyString() {
989-
converter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, ""), false);
991+
converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, ""), false);
990992
assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes()));
991993
}
992994

993995
@Test
994996
public void testSchemaContentValidSchema() {
995-
converter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"type\": \"string\" }"), false);
997+
converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"type\": \"string\" }"), false);
996998
assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectData(TOPIC, "\"foo-bar-baz\"".getBytes()));
997999
}
9981000

9991001
@Test
10001002
public void testSchemaContentInValidSchema() {
10011003
assertThrows(
10021004
DataException.class,
1003-
() -> converter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"string\" }"), false),
1005+
() -> converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"string\" }"), false),
10041006
" Provided schema is invalid , please recheck the schema you have provided");
10051007
}
10061008

10071009
@Test
10081010
public void testSchemaContentLooksLikeSchema() {
1009-
converter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"type\": \"struct\", \"fields\": [{\"field\": \"schema\", \"type\": \"struct\",\"fields\": [{\"field\": \"type\", \"type\": \"string\" }]}, {\"field\": \"payload\", \"type\": \"string\"}]}"), false);
1011+
converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"type\": \"struct\", \"fields\": [{\"field\": \"schema\", \"type\": \"struct\",\"fields\": [{\"field\": \"type\", \"type\": \"string\" }]}, {\"field\": \"payload\", \"type\": \"string\"}]}"), false);
10101012
SchemaAndValue connectData = converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes());
10111013
assertEquals("foo-bar-baz", ((Struct) connectData.value()).getString("payload"));
10121014
}
10131015

1016+
@ParameterizedTest
1017+
@ValueSource(strings = {
1018+
"{ }",
1019+
"{ \"wrong\": \"schema\" }",
1020+
"{ \"schema\": { \"type\": \"string\" } }",
1021+
"{ \"payload\": \"foo-bar-baz\" }",
1022+
"{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\", \"extra\": \"field\" }",
1023+
})
1024+
public void testNullSchemaContentWithWrongConnectDataValue(String value) {
1025+
converter.configure(Map.of(), false);
1026+
assertThrows(
1027+
DataException.class,
1028+
() -> converter.toConnectData(TOPIC, value.getBytes()));
1029+
}
1030+
10141031
private JsonNode parse(byte[] json) {
10151032
try {
10161033
return objectMapper.readTree(json);

0 commit comments

Comments
 (0)