Skip to content

Commit

Permalink
fix: decimals in structs should display as numeric (#4165)
Browse files Browse the repository at this point in the history
  • Loading branch information
agavra authored Dec 18, 2019
1 parent 6d769ad commit 75b539e
Show file tree
Hide file tree
Showing 2 changed files with 14 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,12 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableMap;
import java.io.IOException;
import java.util.Collections;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.json.DecimalFormat;
import org.apache.kafka.connect.json.JsonConverter;
import org.apache.kafka.connect.json.JsonConverterConfig;

public class StructSerializationModule extends SimpleModule {

Expand All @@ -32,7 +34,10 @@ public class StructSerializationModule extends SimpleModule {

public StructSerializationModule() {
super();
jsonConverter.configure(Collections.singletonMap("schemas.enable", false), false);
jsonConverter.configure(ImmutableMap.of(
JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, false,
JsonConverterConfig.DECIMAL_FORMAT_CONFIG, DecimalFormat.NUMERIC.name()
), false);
addSerializer(Struct.class, new StructSerializationModule.Serializer());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.confluent.ksql.util.DecimalUtil;
import java.math.BigDecimal;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
Expand Down Expand Up @@ -48,6 +50,7 @@ public class StructSerializationModuleTest {
.field("ITEMID", Schema.INT64_SCHEMA)
.field("NAME", Schema.STRING_SCHEMA)
.field("CATEGORY", categorySchema)
.field("COST", DecimalUtil.builder(4, 2).build())
.optional().build();

private ObjectMapper objectMapper;
Expand Down Expand Up @@ -82,9 +85,10 @@ public void shouldSerializeStructWithNestedStructCorrectly() throws JsonProcessi
item.put("ITEMID", 1L);
item.put("NAME", "ICE CREAM");
item.put("CATEGORY", category);
item.put("COST", new BigDecimal("10.01"));
final byte[] serializedBytes = objectMapper.writeValueAsBytes(item);
final String jsonString = new String(serializedBytes, StandardCharsets.UTF_8);
assertThat(jsonString, equalTo("{\"ITEMID\":1,\"NAME\":\"ICE CREAM\",\"CATEGORY\":{\"ID\":1,\"NAME\":\"Food\"}}"));
assertThat(jsonString, equalTo("{\"ITEMID\":1,\"NAME\":\"ICE CREAM\",\"CATEGORY\":{\"ID\":1,\"NAME\":\"Food\"},\"COST\":10.01}"));
}

@Test
Expand All @@ -99,7 +103,7 @@ public void shouldSerializeStructWithNestedStructAndNullFieldsCorrectly() throws
item.put("CATEGORY", null);
final byte[] serializedBytes = objectMapper.writeValueAsBytes(item);
final String jsonString = new String(serializedBytes, StandardCharsets.UTF_8);
assertThat(jsonString, equalTo("{\"ITEMID\":1,\"NAME\":\"ICE CREAM\",\"CATEGORY\":null}"));
assertThat(jsonString, equalTo("{\"ITEMID\":1,\"NAME\":\"ICE CREAM\",\"CATEGORY\":null,\"COST\":null}"));
}

@Test
Expand All @@ -122,6 +126,6 @@ public void shouldSerializeStructInsideListCorrectly() throws JsonProcessingExce

final byte[] serializedBytes = objectMapper.writeValueAsBytes(list);
final String jsonString = new String(serializedBytes, StandardCharsets.UTF_8);
assertThat(jsonString, equalTo("[\"Hello\",1,1,1.0,{\"ITEMID\":1,\"NAME\":\"ICE CREAM\",\"CATEGORY\":null}]"));
assertThat(jsonString, equalTo("[\"Hello\",1,1,1.0,{\"ITEMID\":1,\"NAME\":\"ICE CREAM\",\"CATEGORY\":null,\"COST\":null}]"));
}
}

0 comments on commit 75b539e

Please sign in to comment.