Skip to content

Commit

Permalink
feat: support numeric json serde for decimals (#3588)
Browse files Browse the repository at this point in the history
  • Loading branch information
agavra authored Oct 24, 2019
1 parent a8a3588 commit 8621594
Show file tree
Hide file tree
Showing 29 changed files with 586 additions and 260 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,5 +31,6 @@ public enum JsonMapper {
mapper.registerModule(new StructSerializationModule());
mapper.registerModule(new KsqlTypesSerializationModule());
mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES);
mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -148,10 +148,12 @@ public static BigDecimal ensureFit(final BigDecimal value, final Schema schema)
} catch (final ArithmeticException e) {
throw new KsqlException(
String.format(
"Cannot fit decimal '%s' into DECIMAL(%d, %d) without rounding.",
"Cannot fit decimal '%s' into DECIMAL(%d, %d) without rounding. (Requires %d,%d)",
value.toPlainString(),
precision,
scale));
scale,
value.precision(),
value.scale()));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ void sendRecord(final ProducerRecord<byte[], byte[]> record) {
stubKafkaService.getTopic(record.topic()),
new String(record.key(), StandardCharsets.UTF_8),
value,
null,
Optional.of(record.timestamp()),
null
),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import static java.nio.charset.StandardCharsets.UTF_8;

import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
import io.confluent.ksql.test.TestFrameworkException;
Expand Down Expand Up @@ -45,7 +46,8 @@ public final class JsonTestLoader<T extends Test> implements TestLoader<T> {
// mvn test -pl ksql-engine -Dtest=QueryTranslationTest -Dksql.test.files=test1.json,test2,json
private static final String KSQL_TEST_FILES = "ksql.test.files";

private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().enable(
DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS);

static {
OBJECT_MAPPER.registerModule(new Jdk8Module());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ public Record build(final Map<String, Topic> topics) {
topic,
key.orElse(null),
recordValue,
value,
timestamp,
window.orElse(null)
);
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import io.confluent.kafka.serializers.KafkaAvroDeserializer;
import io.confluent.kafka.serializers.KafkaAvroSerializer;
import io.confluent.ksql.test.serde.SerdeSupplier;
import io.confluent.ksql.test.serde.ValueSpec;
import io.confluent.ksql.util.KsqlConstants;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.KsqlPreconditions;
Expand Down Expand Up @@ -191,9 +190,6 @@ private static Object valueSpecUnionToAvro(
@SuppressWarnings("unchecked")
private static GenericRecord getAvroRecord(final Object spec, final Schema schema) {
final GenericRecord record = new GenericData.Record(schema);
if (spec instanceof ValueSpec) {
return getAvroRecord(((ValueSpec) spec).getSpec(), schema);
}
final Map<String, String> caseInsensitiveFieldNames
= getUppercaseKeyToActualKey((Map) spec);
for (final org.apache.avro.Schema.Field field : schema.getFields()) {
Expand Down Expand Up @@ -276,11 +272,10 @@ public Object deserialize(final String topicName, final byte[] data) {
} catch (final Exception e) {
throw new RuntimeException(e);
}
return new ValueSpec(
avroToValueSpec(
avroObject,
new org.apache.avro.Schema.Parser().parse(schemaString),
false));
return avroToValueSpec(
avroObject,
new org.apache.avro.Schema.Parser().parse(schemaString),
false);
}

// CHECKSTYLE_RULES.OFF: CyclomaticComplexity
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

package io.confluent.ksql.test.serde.json;

import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
Expand All @@ -34,6 +35,9 @@

public class ValueSpecJsonSerdeSupplier implements SerdeSupplier<Object> {

private static final ObjectMapper MAPPER = new ObjectMapper()
.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS);

@Override
public Serializer<Object> getSerializer(final SchemaRegistryClient schemaRegistryClient) {
return new ValueSpecJsonSerializer();
Expand All @@ -60,7 +64,7 @@ public byte[] serialize(final String topicName, final Object spec) {
}
try {
final Object toSerialize = Converter.toJsonNode(spec);
return new ObjectMapper().writeValueAsBytes(toSerialize);
return MAPPER.writeValueAsBytes(toSerialize);
} catch (final Exception e) {
throw new RuntimeException(e);
}
Expand All @@ -82,7 +86,7 @@ public Object deserialize(final String topicName, final byte[] data) {
return null;
}
try {
return new ObjectMapper().readValue(data, Object.class);
return MAPPER.readValue(data, Object.class);
} catch (final Exception e) {
throw new RuntimeException(e);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,157 @@
/*
* Copyright 2019 Confluent Inc.
*
* Licensed under the Confluent Community License (the "License"; you may not use
* this file except in compliance with the License. You may obtain a copy of the
* License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.test.tools;

import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.JsonNodeType;
import com.fasterxml.jackson.databind.node.NumericNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.fasterxml.jackson.databind.node.TextNode;
import com.google.common.collect.ImmutableMap;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.function.BiPredicate;
import java.util.function.Function;
import org.apache.kafka.connect.data.Struct;

public final class ExpectedRecordComparator {

private static final Map<JsonNodeType, BiPredicate<Object, JsonNode>> COMPARATORS =
ImmutableMap.<JsonNodeType, BiPredicate<Object, JsonNode>>builder()
.put(JsonNodeType.OBJECT, ExpectedRecordComparator::compareStruct)
.put(JsonNodeType.ARRAY, ExpectedRecordComparator::compareArray)
.put(JsonNodeType.NUMBER, ExpectedRecordComparator::compareNumber)
.put(JsonNodeType.STRING, ExpectedRecordComparator::compareText)
.put(JsonNodeType.BOOLEAN, ExpectedRecordComparator::compareBoolean)
.put(JsonNodeType.NULL, ExpectedRecordComparator::compareNull)
.build();

private ExpectedRecordComparator() {
}

public static boolean matches(final Object actualValue, final JsonNode expectedValue) {
return comparator(expectedValue).test(actualValue, expectedValue);
}

private static boolean compareStruct(final Object actualValue, final JsonNode expectedValue) {
final ObjectNode expected = (ObjectNode) expectedValue;
final Function<String, Object> getter;
if (actualValue instanceof Struct) {
getter = ((Struct) actualValue)::get;
} else if (actualValue instanceof Map) {
getter = ((Map<?, ?>) actualValue)::get;
} else {
return false;
}

final Iterator<Entry<String, JsonNode>> fields = expected.fields();
while (fields.hasNext()) {
final Entry<String, JsonNode> field = fields.next();
if (!comparator(field.getValue()).test(getter.apply(field.getKey()), field.getValue())) {
return false;
}
}
return true;
}

private static boolean compareArray(final Object actualValue, final JsonNode expectedValue) {
final ArrayNode expected = (ArrayNode) expectedValue;
if (actualValue instanceof List) {
final List<?> actual = (List<?>) actualValue;
final Iterator<JsonNode> elements = expected.elements();

int i = 0;
while (elements.hasNext()) {
final JsonNode el = elements.next();
if (!comparator(el).test(actual.get(i), el)) {
return false;
}
i++;
}

return true;
}
return false;
}

private static boolean compareNumber(final Object actualValue, final JsonNode expectedValue) {
final NumericNode expected = (NumericNode) expectedValue;
if (actualValue instanceof Integer) {
return expected.intValue() == (Integer) actualValue;
}
if (actualValue instanceof Long) {
return expected.longValue() == (Long) actualValue;
}
if (actualValue instanceof Double) {
return expected.doubleValue() == (Double) actualValue;
}
if (actualValue instanceof BigDecimal) {
if (!expected.isBigDecimal()) {
// we don't want to risk comparing a BigDecimal with something of
// lower precision
return false;
}

expected.isBigDecimal();
try {
return expected.decimalValue()
.setScale(((BigDecimal) actualValue).scale(), RoundingMode.UNNECESSARY)
.equals(actualValue);
} catch (final ArithmeticException e) {
// the scale of the expected value cannot match the scale of the actual value
// without rounding
return false;
}
}
return false;
}

private static boolean compareText(final Object actualValue, final JsonNode expectedValue) {
final TextNode expected = (TextNode) expectedValue;
if (actualValue instanceof String) {
return expected.asText().equals(actualValue);
}
if (actualValue instanceof BigDecimal) {
return new BigDecimal(expected.asText()).equals(actualValue);
}
return false;
}

private static boolean compareBoolean(final Object actualValue, final JsonNode expectedValue) {
return expectedValue.asBoolean() == (Boolean) actualValue;
}

private static boolean compareNull(final Object actualValue, final JsonNode expectedValue) {
return actualValue == null;
}

private static BiPredicate<Object, JsonNode> comparator(final JsonNode node) {
final JsonNodeType type = node == null ? JsonNodeType.NULL : node.getNodeType();
final BiPredicate<Object, JsonNode> predicate = COMPARATORS.get(type);

if (predicate == null) {
throw new IllegalArgumentException(
"KSQL Testing Tool cannot expect JSON node type: " + type);
}
return predicate;
}

}
Loading

0 comments on commit 8621594

Please sign in to comment.