new conversion tests added
This commit is contained in:
parent
6367f44a90
commit
152b088771
3 changed files with 567 additions and 357 deletions
|
@ -189,11 +189,8 @@ public class JsonAvroConversion {
|
||||||
yield node;
|
yield node;
|
||||||
}
|
}
|
||||||
case MAP -> {
|
case MAP -> {
|
||||||
var map = (Map<Utf8, Object>) obj;
|
|
||||||
ObjectNode node = MAPPER.createObjectNode();
|
ObjectNode node = MAPPER.createObjectNode();
|
||||||
map.forEach((k, v) -> {
|
((Map) obj).forEach((k, v) -> node.set(k.toString(), convertAvroToJson(v, avroSchema.getValueType())));
|
||||||
node.set(k.toString(), convertAvroToJson(v, avroSchema.getValueType()));
|
|
||||||
});
|
|
||||||
yield node;
|
yield node;
|
||||||
}
|
}
|
||||||
case ARRAY -> {
|
case ARRAY -> {
|
||||||
|
@ -206,8 +203,6 @@ public class JsonAvroConversion {
|
||||||
yield new TextNode(obj.toString());
|
yield new TextNode(obj.toString());
|
||||||
}
|
}
|
||||||
case UNION -> {
|
case UNION -> {
|
||||||
//TODO: cover with tests
|
|
||||||
// non-null case
|
|
||||||
ObjectNode node = MAPPER.createObjectNode();
|
ObjectNode node = MAPPER.createObjectNode();
|
||||||
int unionIdx = AvroData.getGenericData().resolveUnion(avroSchema, obj);
|
int unionIdx = AvroData.getGenericData().resolveUnion(avroSchema, obj);
|
||||||
Schema unionType = avroSchema.getTypes().get(unionIdx);
|
Schema unionType = avroSchema.getTypes().get(unionIdx);
|
||||||
|
@ -232,25 +227,17 @@ public class JsonAvroConversion {
|
||||||
}
|
}
|
||||||
yield new IntNode((Integer) obj);
|
yield new IntNode((Integer) obj);
|
||||||
}
|
}
|
||||||
case FLOAT -> {
|
case FLOAT -> new FloatNode((Float) obj);
|
||||||
yield new FloatNode((Float) obj);
|
case DOUBLE -> new DoubleNode((Double) obj);
|
||||||
}
|
case BOOLEAN -> BooleanNode.valueOf((Boolean) obj);
|
||||||
case DOUBLE -> {
|
case NULL -> NullNode.getInstance();
|
||||||
yield new DoubleNode((Double) obj);
|
|
||||||
}
|
|
||||||
case BOOLEAN -> {
|
|
||||||
yield BooleanNode.valueOf((Boolean) obj);
|
|
||||||
}
|
|
||||||
case NULL -> {
|
|
||||||
yield NullNode.getInstance();
|
|
||||||
}
|
|
||||||
case BYTES -> {
|
case BYTES -> {
|
||||||
if (isLogicalType(avroSchema)) {
|
if (isLogicalType(avroSchema)) {
|
||||||
yield processLogicalType(obj, avroSchema);
|
yield processLogicalType(obj, avroSchema);
|
||||||
}
|
}
|
||||||
//TODO: check with tests
|
//TODO: check with tests
|
||||||
byte[] bytes = (byte[]) obj;
|
ByteBuffer bytes = (ByteBuffer) obj;
|
||||||
yield new TextNode(new String(bytes)); //TODO: encoding
|
yield new TextNode(new String(bytes.array()));
|
||||||
}
|
}
|
||||||
case FIXED -> {
|
case FIXED -> {
|
||||||
if (isLogicalType(avroSchema)) {
|
if (isLogicalType(avroSchema)) {
|
||||||
|
@ -263,27 +250,26 @@ public class JsonAvroConversion {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Object processLogicalType(JsonNode node, Schema schema) {
|
private static Object processLogicalType(JsonNode node, Schema schema) {
|
||||||
String logicalTypeName = schema.getLogicalType().getName();
|
return findConversion(schema)
|
||||||
var conversion = Stream.of(LogicalTypeConversion.values())
|
|
||||||
.filter(t -> t.name.equalsIgnoreCase(logicalTypeName))
|
|
||||||
.findFirst();
|
|
||||||
return conversion
|
|
||||||
.map(c -> c.jsonToAvroConversion.apply(node, schema))
|
.map(c -> c.jsonToAvroConversion.apply(node, schema))
|
||||||
.orElseThrow(() ->
|
.orElseThrow(() ->
|
||||||
new JsonToAvroConversionException("'%s' logical type is not supported"
|
new JsonToAvroConversionException("'%s' logical type is not supported"
|
||||||
.formatted(logicalTypeName)));
|
.formatted(schema.getLogicalType().getName())));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static JsonNode processLogicalType(Object obj, Schema schema) {
|
private static JsonNode processLogicalType(Object obj, Schema schema) {
|
||||||
String logicalTypeName = schema.getLogicalType().getName();
|
return findConversion(schema)
|
||||||
var conversion = Stream.of(LogicalTypeConversion.values())
|
|
||||||
.filter(t -> t.name.equalsIgnoreCase(logicalTypeName))
|
|
||||||
.findFirst();
|
|
||||||
return conversion
|
|
||||||
.map(c -> c.avroToJsonConversion.apply(obj, schema))
|
.map(c -> c.avroToJsonConversion.apply(obj, schema))
|
||||||
.orElseThrow(() ->
|
.orElseThrow(() ->
|
||||||
new JsonToAvroConversionException("'%s' logical type is not supported"
|
new JsonToAvroConversionException("'%s' logical type is not supported"
|
||||||
.formatted(logicalTypeName)));
|
.formatted(schema.getLogicalType().getName())));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Optional<LogicalTypeConversion> findConversion(Schema schema) {
|
||||||
|
String logicalTypeName = schema.getLogicalType().getName();
|
||||||
|
return Stream.of(LogicalTypeConversion.values())
|
||||||
|
.filter(t -> t.name.equalsIgnoreCase(logicalTypeName))
|
||||||
|
.findFirst();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static boolean isLogicalType(Schema schema) {
|
private static boolean isLogicalType(Schema schema) {
|
||||||
|
|
|
@ -6,8 +6,8 @@ import com.fasterxml.jackson.databind.json.JsonMapper;
|
||||||
import com.provectus.kafka.ui.serde.api.DeserializeResult;
|
import com.provectus.kafka.ui.serde.api.DeserializeResult;
|
||||||
import com.provectus.kafka.ui.serde.api.SchemaDescription;
|
import com.provectus.kafka.ui.serde.api.SchemaDescription;
|
||||||
import com.provectus.kafka.ui.serde.api.Serde;
|
import com.provectus.kafka.ui.serde.api.Serde;
|
||||||
|
import com.provectus.kafka.ui.util.jsonschema.JsonAvroConversion;
|
||||||
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
|
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
|
||||||
import io.confluent.kafka.schemaregistry.avro.AvroSchemaUtils;
|
|
||||||
import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient;
|
import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient;
|
||||||
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
|
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
|
||||||
import java.io.ByteArrayOutputStream;
|
import java.io.ByteArrayOutputStream;
|
||||||
|
@ -212,13 +212,13 @@ class SchemaRegistrySerdeTest {
|
||||||
GenericDatumWriter<Object> writer = new GenericDatumWriter<>(schema.rawSchema());
|
GenericDatumWriter<Object> writer = new GenericDatumWriter<>(schema.rawSchema());
|
||||||
ByteArrayOutputStream output = new ByteArrayOutputStream();
|
ByteArrayOutputStream output = new ByteArrayOutputStream();
|
||||||
Encoder encoder = EncoderFactory.get().binaryEncoder(output, null);
|
Encoder encoder = EncoderFactory.get().binaryEncoder(output, null);
|
||||||
writer.write(AvroSchemaUtils.toObject(json, schema), encoder);
|
writer.write(JsonAvroConversion.convertJsonToAvro(json, schema.rawSchema()), encoder);
|
||||||
encoder.flush();
|
encoder.flush();
|
||||||
return output.toByteArray();
|
return output.toByteArray();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void fieldsRepresentationIsConsistentForSerializationAndDeserialization() throws Exception {
|
void avroFieldsRepresentationIsConsistentForSerializationAndDeserialization() throws Exception {
|
||||||
AvroSchema schema = new AvroSchema(
|
AvroSchema schema = new AvroSchema(
|
||||||
"""
|
"""
|
||||||
{
|
{
|
||||||
|
@ -268,6 +268,10 @@ class SchemaRegistrySerdeTest {
|
||||||
{
|
{
|
||||||
"name": "f_union",
|
"name": "f_union",
|
||||||
"type": ["null", "string", "int" ]
|
"type": ["null", "string", "int" ]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "f_optional_to_test_not_filled_case",
|
||||||
|
"type": [ "null", "string"]
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
}"""
|
}"""
|
||||||
|
@ -288,18 +292,11 @@ class SchemaRegistrySerdeTest {
|
||||||
""";
|
""";
|
||||||
|
|
||||||
registryClient.register("test-value", schema);
|
registryClient.register("test-value", schema);
|
||||||
|
assertSerdeCycle("test", jsonPayload);
|
||||||
byte[] serializedBytes = serde.serializer("test", Serde.Target.VALUE).serialize(jsonPayload);
|
|
||||||
|
|
||||||
var deserializedJson = serde.deserializer("test", Serde.Target.VALUE)
|
|
||||||
.deserialize(null, serializedBytes)
|
|
||||||
.getResult();
|
|
||||||
|
|
||||||
assertJsonsEqual(jsonPayload, deserializedJson);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void logicalTypesRepresentationIsConsistentForSerializationAndDeserialization() throws Exception {
|
void avroLogicalTypesRepresentationIsConsistentForSerializationAndDeserialization() throws Exception {
|
||||||
AvroSchema schema = new AvroSchema(
|
AvroSchema schema = new AvroSchema(
|
||||||
"""
|
"""
|
||||||
{
|
{
|
||||||
|
@ -361,14 +358,18 @@ class SchemaRegistrySerdeTest {
|
||||||
""";
|
""";
|
||||||
|
|
||||||
registryClient.register("test-value", schema);
|
registryClient.register("test-value", schema);
|
||||||
|
assertSerdeCycle("test", jsonPayload);
|
||||||
|
}
|
||||||
|
|
||||||
byte[] serializedBytes = serde.serializer("test", Serde.Target.VALUE).serialize(jsonPayload);
|
// 1. serialize input json to binary
|
||||||
|
// 2. deserialize from binary
|
||||||
var deserializedJson = serde.deserializer("test", Serde.Target.VALUE)
|
// 3. check that deserialized version equal to input
|
||||||
|
void assertSerdeCycle(String topic, String jsonInput) {
|
||||||
|
byte[] serializedBytes = serde.serializer(topic, Serde.Target.VALUE).serialize(jsonInput);
|
||||||
|
var deserializedJson = serde.deserializer(topic, Serde.Target.VALUE)
|
||||||
.deserialize(null, serializedBytes)
|
.deserialize(null, serializedBytes)
|
||||||
.getResult();
|
.getResult();
|
||||||
|
assertJsonsEqual(jsonInput, deserializedJson);
|
||||||
assertJsonsEqual(jsonPayload, deserializedJson);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,8 +1,17 @@
|
||||||
package com.provectus.kafka.ui.util.jsonschema;
|
package com.provectus.kafka.ui.util.jsonschema;
|
||||||
|
|
||||||
import static com.provectus.kafka.ui.util.jsonschema.JsonAvroConversion.convertJsonToAvro;
|
import static com.provectus.kafka.ui.util.jsonschema.JsonAvroConversion.convertJsonToAvro;
|
||||||
|
import static com.provectus.kafka.ui.util.jsonschema.JsonAvroConversion.convertAvroToJson;
|
||||||
import static org.assertj.core.api.Assertions.assertThat;
|
import static org.assertj.core.api.Assertions.assertThat;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
|
import com.fasterxml.jackson.databind.json.JsonMapper;
|
||||||
|
import com.fasterxml.jackson.databind.node.BooleanNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.DoubleNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.FloatNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.IntNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.LongNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.TextNode;
|
||||||
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
|
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
|
||||||
import java.math.BigDecimal;
|
import java.math.BigDecimal;
|
||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
|
@ -11,12 +20,19 @@ import java.time.LocalDateTime;
|
||||||
import java.time.LocalTime;
|
import java.time.LocalTime;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.UUID;
|
||||||
|
import lombok.SneakyThrows;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericData;
|
import org.apache.avro.generic.GenericData;
|
||||||
|
import org.junit.jupiter.api.Nested;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
class JsonAvroConversionTest {
|
class JsonAvroConversionTest {
|
||||||
|
|
||||||
|
// checking conversion from json to KafkaAvroSerializer-compatible avro objects
|
||||||
|
@Nested
|
||||||
|
class FromJsonToAvro {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void primitiveRoot() {
|
void primitiveRoot() {
|
||||||
assertThat(convertJsonToAvro("\"str\"", createSchema("\"string\"")))
|
assertThat(convertJsonToAvro("\"str\"", createSchema("\"string\"")))
|
||||||
|
@ -112,15 +128,15 @@ class JsonAvroConversionTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void unionRoot() {
|
void unionRoot() {
|
||||||
var sc = createSchema("[ \"null\", \"string\", \"int\" ]");
|
var schema = createSchema("[ \"null\", \"string\", \"int\" ]");
|
||||||
|
|
||||||
var converted = convertJsonToAvro("{\"string\":\"string here\"}", sc);
|
var converted = convertJsonToAvro("{\"string\":\"string here\"}", schema);
|
||||||
assertThat(converted).isEqualTo("string here");
|
assertThat(converted).isEqualTo("string here");
|
||||||
|
|
||||||
converted = convertJsonToAvro("{\"int\": 123}", sc);
|
converted = convertJsonToAvro("{\"int\": 123}", schema);
|
||||||
assertThat(converted).isEqualTo(123);
|
assertThat(converted).isEqualTo(123);
|
||||||
|
|
||||||
converted = convertJsonToAvro("null", sc);
|
converted = convertJsonToAvro("null", schema);
|
||||||
assertThat(converted).isEqualTo(null);
|
assertThat(converted).isEqualTo(null);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -130,6 +146,7 @@ class JsonAvroConversionTest {
|
||||||
"""
|
"""
|
||||||
{
|
{
|
||||||
"type": "record",
|
"type": "record",
|
||||||
|
"namespace": "com.test",
|
||||||
"name": "TestAvroRecord",
|
"name": "TestAvroRecord",
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
|
@ -142,20 +159,16 @@ class JsonAvroConversionTest {
|
||||||
|
|
||||||
String jsonPayload = "{ \"f_union\": null }";
|
String jsonPayload = "{ \"f_union\": null }";
|
||||||
|
|
||||||
var converted = convertJsonToAvro(jsonPayload, schema);
|
var record = (GenericData.Record) convertJsonToAvro(jsonPayload, schema);
|
||||||
assertThat(converted).isInstanceOf(GenericData.Record.class);
|
|
||||||
|
|
||||||
var record = (GenericData.Record) converted;
|
|
||||||
assertThat(record.get("f_union")).isNull();
|
assertThat(record.get("f_union")).isNull();
|
||||||
|
|
||||||
|
|
||||||
jsonPayload = "{ \"f_union\": { \"int\": 123 } }";
|
jsonPayload = "{ \"f_union\": { \"int\": 123 } }";
|
||||||
record = (GenericData.Record) convertJsonToAvro(jsonPayload, schema);
|
record = (GenericData.Record) convertJsonToAvro(jsonPayload, schema);
|
||||||
assertThat(record.get("f_union")).isEqualTo(123);
|
assertThat(record.get("f_union")).isEqualTo(123);
|
||||||
|
|
||||||
jsonPayload = "{ \"f_union\": { \"TestAvroRecord\": { \"f_union\": { \"int\": 123 } } } }";
|
//inner-record's name should be fully-qualified!
|
||||||
|
jsonPayload = "{ \"f_union\": { \"com.test.TestAvroRecord\": { \"f_union\": { \"int\": 123 } } } }";
|
||||||
record = (GenericData.Record) convertJsonToAvro(jsonPayload, schema);
|
record = (GenericData.Record) convertJsonToAvro(jsonPayload, schema);
|
||||||
|
|
||||||
assertThat(record.get("f_union")).isInstanceOf(GenericData.Record.class);
|
assertThat(record.get("f_union")).isInstanceOf(GenericData.Record.class);
|
||||||
var innerRec = (GenericData.Record) record.get("f_union");
|
var innerRec = (GenericData.Record) record.get("f_union");
|
||||||
assertThat(innerRec.get("f_union")).isEqualTo(123);
|
assertThat(innerRec.get("f_union")).isEqualTo(123);
|
||||||
|
@ -216,10 +229,7 @@ class JsonAvroConversionTest {
|
||||||
}
|
}
|
||||||
""";
|
""";
|
||||||
|
|
||||||
var converted = convertJsonToAvro(jsonPayload, schema);
|
var record = (GenericData.Record) convertJsonToAvro(jsonPayload, schema);
|
||||||
assertThat(converted).isInstanceOf(GenericData.Record.class);
|
|
||||||
|
|
||||||
var record = (GenericData.Record) converted;
|
|
||||||
assertThat(record.get("long_map"))
|
assertThat(record.get("long_map"))
|
||||||
.isEqualTo(Map.of("k1", 123L, "k2", 456L));
|
.isEqualTo(Map.of("k1", 123L, "k2", 456L));
|
||||||
assertThat(record.get("string_map"))
|
assertThat(record.get("string_map"))
|
||||||
|
@ -266,10 +276,7 @@ class JsonAvroConversionTest {
|
||||||
}
|
}
|
||||||
""";
|
""";
|
||||||
|
|
||||||
var converted = convertJsonToAvro(jsonPayload, schema);
|
var record = (GenericData.Record) convertJsonToAvro(jsonPayload, schema);
|
||||||
assertThat(converted).isInstanceOf(GenericData.Record.class);
|
|
||||||
|
|
||||||
var record = (GenericData.Record) converted;
|
|
||||||
assertThat(record.get("f_array")).isEqualTo(List.of("e1", "e2"));
|
assertThat(record.get("f_array")).isEqualTo(List.of("e1", "e2"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -357,9 +364,225 @@ class JsonAvroConversionTest {
|
||||||
assertThat(record.get("lt_local_timestamp_micros"))
|
assertThat(record.get("lt_local_timestamp_micros"))
|
||||||
.isEqualTo(LocalDateTime.parse("2017-12-13T10:15:30.123456"));
|
.isEqualTo(LocalDateTime.parse("2017-12-13T10:15:30.123456"));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// checking conversion of KafkaAvroDeserializer output to JsonNode
|
||||||
|
@Nested
|
||||||
|
class FromAvroToJson {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void primitiveRoot() {
|
||||||
|
assertThat(convertAvroToJson("str", createSchema("\"string\"")))
|
||||||
|
.isEqualTo(new TextNode("str"));
|
||||||
|
|
||||||
|
assertThat(convertAvroToJson(123, createSchema("\"int\"")))
|
||||||
|
.isEqualTo(new IntNode(123));
|
||||||
|
|
||||||
|
assertThat(convertAvroToJson(123L, createSchema("\"long\"")))
|
||||||
|
.isEqualTo(new LongNode(123));
|
||||||
|
|
||||||
|
assertThat(convertAvroToJson(123.1F, createSchema("\"float\"")))
|
||||||
|
.isEqualTo(new FloatNode(123.1F));
|
||||||
|
|
||||||
|
assertThat(convertAvroToJson(123.1, createSchema("\"double\"")))
|
||||||
|
.isEqualTo(new DoubleNode(123.1));
|
||||||
|
|
||||||
|
assertThat(convertAvroToJson(true, createSchema("\"boolean\"")))
|
||||||
|
.isEqualTo(BooleanNode.valueOf(true));
|
||||||
|
}
|
||||||
|
|
||||||
|
@SneakyThrows
|
||||||
|
@Test
|
||||||
|
void primitiveTypedFields() {
|
||||||
|
var schema = createSchema(
|
||||||
|
"""
|
||||||
|
{
|
||||||
|
"type": "record",
|
||||||
|
"name": "TestAvroRecord",
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"name": "f_int",
|
||||||
|
"type": "int"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "f_long",
|
||||||
|
"type": "long"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "f_string",
|
||||||
|
"type": "string"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "f_boolean",
|
||||||
|
"type": "boolean"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "f_float",
|
||||||
|
"type": "float"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "f_double",
|
||||||
|
"type": "double"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "f_enum",
|
||||||
|
"type" : {
|
||||||
|
"type": "enum",
|
||||||
|
"name": "Suit",
|
||||||
|
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}"""
|
||||||
|
);
|
||||||
|
|
||||||
|
GenericData.Record inputRecord = new GenericData.Record(schema);
|
||||||
|
inputRecord.put("f_int", 123);
|
||||||
|
inputRecord.put("f_long", 4294967294L);
|
||||||
|
inputRecord.put("f_string", "string here");
|
||||||
|
inputRecord.put("f_boolean", true);
|
||||||
|
inputRecord.put("f_float", 123.1f);
|
||||||
|
inputRecord.put("f_double", 123456.123456);
|
||||||
|
inputRecord.put("f_enum", new GenericData.EnumSymbol(schema.getField("f_enum").schema(), "SPADES"));
|
||||||
|
|
||||||
|
String expectedJson = """
|
||||||
|
{
|
||||||
|
"f_int": 123,
|
||||||
|
"f_long": 4294967294,
|
||||||
|
"f_string": "string here",
|
||||||
|
"f_boolean": true,
|
||||||
|
"f_float": 123.1,
|
||||||
|
"f_double": 123456.123456,
|
||||||
|
"f_enum": "SPADES"
|
||||||
|
}
|
||||||
|
""";
|
||||||
|
|
||||||
|
assertJsonsEqual(expectedJson, convertAvroToJson(inputRecord, schema));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void logicalTypesField() {
|
||||||
|
var schema = createSchema(
|
||||||
|
"""
|
||||||
|
{
|
||||||
|
"type": "record",
|
||||||
|
"name": "TestAvroRecord",
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"name": "lt_date",
|
||||||
|
"type": { "type": "int", "logicalType": "date" }
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lt_uuid",
|
||||||
|
"type": { "type": "string", "logicalType": "uuid" }
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lt_decimal",
|
||||||
|
"type": { "type": "bytes", "logicalType": "decimal", "precision": 22, "scale":10 }
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lt_time_millis",
|
||||||
|
"type": { "type": "int", "logicalType": "time-millis"}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lt_time_micros",
|
||||||
|
"type": { "type": "long", "logicalType": "time-micros"}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lt_timestamp_millis",
|
||||||
|
"type": { "type": "long", "logicalType": "timestamp-millis" }
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lt_timestamp_micros",
|
||||||
|
"type": { "type": "long", "logicalType": "timestamp-micros" }
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lt_local_timestamp_millis",
|
||||||
|
"type": { "type": "long", "logicalType": "local-timestamp-millis" }
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lt_local_timestamp_micros",
|
||||||
|
"type": { "type": "long", "logicalType": "local-timestamp-micros" }
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}"""
|
||||||
|
);
|
||||||
|
|
||||||
|
GenericData.Record inputRecord = new GenericData.Record(schema);
|
||||||
|
inputRecord.put("lt_date", LocalDate.of(1991, 8, 14));
|
||||||
|
inputRecord.put("lt_decimal", new BigDecimal("2.1617413862327545E11"));
|
||||||
|
inputRecord.put("lt_time_millis", LocalTime.parse("10:15:30.001"));
|
||||||
|
inputRecord.put("lt_time_micros", LocalTime.parse("10:15:30.123456"));
|
||||||
|
inputRecord.put("lt_uuid", UUID.fromString("a37b75ca-097c-5d46-6119-f0637922e908"));
|
||||||
|
inputRecord.put("lt_timestamp_millis", Instant.parse("2007-12-03T10:15:30.123Z"));
|
||||||
|
inputRecord.put("lt_timestamp_micros", Instant.parse("2007-12-13T10:15:30.123456Z"));
|
||||||
|
inputRecord.put("lt_local_timestamp_millis", LocalDateTime.parse("2017-12-03T10:15:30.123"));
|
||||||
|
inputRecord.put("lt_local_timestamp_micros", LocalDateTime.parse("2017-12-13T10:15:30.123456"));
|
||||||
|
|
||||||
|
String expectedJson = """
|
||||||
|
{
|
||||||
|
"lt_date":"1991-08-14",
|
||||||
|
"lt_decimal": 2.1617413862327545E11,
|
||||||
|
"lt_time_millis": "10:15:30.001",
|
||||||
|
"lt_time_micros": "10:15:30.123456",
|
||||||
|
"lt_uuid": "a37b75ca-097c-5d46-6119-f0637922e908",
|
||||||
|
"lt_timestamp_millis": "2007-12-03T10:15:30.123Z",
|
||||||
|
"lt_timestamp_micros": "2007-12-13T10:15:30.123456Z",
|
||||||
|
"lt_local_timestamp_millis": "2017-12-03T10:15:30.123",
|
||||||
|
"lt_local_timestamp_micros": "2017-12-13T10:15:30.123456"
|
||||||
|
}
|
||||||
|
""";
|
||||||
|
|
||||||
|
assertJsonsEqual(expectedJson, convertAvroToJson(inputRecord, schema));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void unionField() {
|
||||||
|
var schema = createSchema(
|
||||||
|
"""
|
||||||
|
{
|
||||||
|
"type": "record",
|
||||||
|
"namespace": "com.test",
|
||||||
|
"name": "TestAvroRecord",
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"name": "f_union",
|
||||||
|
"type": [ "null", "int", "TestAvroRecord"]
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}"""
|
||||||
|
);
|
||||||
|
|
||||||
|
var r = new GenericData.Record(schema);
|
||||||
|
r.put("f_union", null);
|
||||||
|
assertJsonsEqual(" {}", convertAvroToJson(r, schema));
|
||||||
|
|
||||||
|
r = new GenericData.Record(schema);
|
||||||
|
r.put("f_union", 123);
|
||||||
|
assertJsonsEqual(" { \"f_union\" : { \"int\" : 123 } }", convertAvroToJson(r, schema));
|
||||||
|
|
||||||
|
|
||||||
|
r = new GenericData.Record(schema);
|
||||||
|
var innerRec = new GenericData.Record(schema);
|
||||||
|
innerRec.put("f_union", 123);
|
||||||
|
r.put("f_union", innerRec);
|
||||||
|
assertJsonsEqual(
|
||||||
|
" { \"f_union\" : { \"com.test.TestAvroRecord\" : { \"f_union\" : { \"int\" : 123 } } } }",
|
||||||
|
convertAvroToJson(r, schema)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
private Schema createSchema(String schema) {
|
private Schema createSchema(String schema) {
|
||||||
return new AvroSchema(schema).rawSchema();
|
return new AvroSchema(schema).rawSchema();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SneakyThrows
|
||||||
|
private void assertJsonsEqual(String expected, JsonNode actual) {
|
||||||
|
var mapper = new JsonMapper();
|
||||||
|
assertThat(actual.toPrettyString())
|
||||||
|
.isEqualTo(mapper.readTree(expected).toPrettyString());
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Add table
Reference in a new issue