private Map<String, Object> enforceFieldTypeForMap( final Schema fieldSchema, final Map<String, ?> columnMap) { final Map<String, Object> ksqlMap = new HashMap<>(); for (final Map.Entry<String, ?> e : columnMap.entrySet()) { ksqlMap.put( enforceFieldType(Schema.OPTIONAL_STRING_SCHEMA, e.getKey()).toString(), enforceFieldType(fieldSchema.valueSchema(), e.getValue()) ); } return ksqlMap; }
@SuppressWarnings("unchecked") private Object enforceFieldType(final Schema fieldSchema, final Object columnVal) { if (columnVal == null) { return null; } switch (fieldSchema.type()) { case BOOLEAN: return SerdeUtils.toBoolean(columnVal); case INT32: return SerdeUtils.toInteger(columnVal); case INT64: return SerdeUtils.toLong(columnVal); case FLOAT64: return SerdeUtils.toDouble(columnVal); case STRING: return processString(columnVal); case ARRAY: return enforceFieldTypeForArray(fieldSchema, (List<?>) columnVal); case MAP: return enforceFieldTypeForMap(fieldSchema, (Map<String, Object>) columnVal); case STRUCT: return enforceFieldTypeForStruct(fieldSchema, (Map<String, Object>) columnVal); default: throw new KsqlException("Type is not supported: " + fieldSchema.type()); } }
private Struct enforceFieldTypeForStruct( final Schema fieldSchema, final Map<String, ?> structMap) { final Struct columnStruct = new Struct(fieldSchema); final Map<String, String> caseInsensitiveStructFieldNameMap = getCaseInsensitiveFieldNameMap(structMap, false); fieldSchema.fields() .forEach( field -> columnStruct.put(field.name(), enforceFieldType( field.schema(), structMap.get( caseInsensitiveStructFieldNameMap.get(field.name().toUpperCase()) )))); return columnStruct; }
@Test public void shouldCreateJsonStringForStructIfDefinedAsVarchar() throws JsonProcessingException { final Schema schema = SchemaBuilder.struct() .field("itemid".toUpperCase(), Schema.OPTIONAL_STRING_SCHEMA) .build(); final KsqlJsonDeserializer deserializer = new KsqlJsonDeserializer( schema, false, recordLogger); final GenericRow expected = new GenericRow(Collections.singletonList( "{\"CATEGORY\":{\"ID\":2,\"NAME\":\"Food\"},\"ITEMID\":6,\"NAME\":\"Item_6\"}")); final GenericRow genericRow = deserializer.deserialize("", "{\"itemid\":{\"CATEGORY\":{\"ID\":2,\"NAME\":\"Food\"},\"ITEMID\":6,\"NAME\":\"Item_6\"}}".getBytes(StandardCharsets.UTF_8)); assertThat(genericRow, equalTo(expected)); }
@Override public Serde<GenericRow> getGenericRowSerde(final Schema schema, final KsqlConfig ksqlConfig, final boolean isInternal, final Supplier<SchemaRegistryClient> schemaRegistryClientFactory, final String loggerNamePrefix) { final Map<String, Object> serdeProps = new HashMap<>(); serdeProps.put("JsonPOJOClass", GenericRow.class); final Serializer<GenericRow> genericRowSerializer = new KsqlJsonSerializer(schema); genericRowSerializer.configure(serdeProps, false); final Deserializer<GenericRow> genericRowDeserializer = new KsqlJsonDeserializer( schema, isInternal, ProcessingLoggerFactory.getLogger( join(loggerNamePrefix, SerdeUtils.DESERIALIZER_LOGGER_NAME)) ); genericRowDeserializer.configure(serdeProps, false); return Serdes.serdeFrom(genericRowSerializer, genericRowDeserializer); } }
@Test public void shouldTreatNullAsNull() throws JsonProcessingException { final Map<String, Object> row = new HashMap<>(); row.put("ordertime", null); row.put("@orderid", null); row.put("itemid", null); row.put("orderunits", null); row.put("arrayCol", new Double[]{0.0, null}); row.put("mapCol", null); final GenericRow expected = new GenericRow(Arrays.asList(null, null, null, null, new Double[]{0.0, null}, null)); final GenericRow genericRow = ksqlJsonDeserializer.deserialize( "", objectMapper.writeValueAsBytes(row)); assertThat(genericRow, equalTo(expected)); }
@Override public GenericRow deserialize(final String topic, final byte[] bytes) { try { final GenericRow row = getGenericRow(bytes); if (LOG.isTraceEnabled()) { LOG.trace("Deserialized row. topic:{}, row:{}", topic, row); } return row; } catch (final Exception e) { recordLogger.error( ProcessingLogMessageFactory.deserializationErrorMsg(e, Optional.ofNullable(bytes))); throw new SerializationException( "KsqlJsonDeserializer failed to deserialize data for topic: " + topic, e); } }
@Test public void shouldDeserializeJsonCorrectlyWithRedundantFields() throws JsonProcessingException { final Map<String, Object> orderRow = new HashMap<>(); orderRow.put("ordertime", 1511897796092L); orderRow.put("@orderid", 1L); orderRow.put("itemid", "Item_1"); orderRow.put("orderunits", 10.0); orderRow.put("arraycol", new Double[]{10.0, 20.0}); orderRow.put("mapcol", Collections.singletonMap("key1", 10.0)); final byte[] jsonBytes = objectMapper.writeValueAsBytes(orderRow); final Schema newOrderSchema = SchemaBuilder.struct() .field("ordertime".toUpperCase(), org.apache.kafka.connect.data.Schema.OPTIONAL_INT64_SCHEMA) .field("orderid".toUpperCase(), org.apache.kafka.connect.data.Schema.OPTIONAL_INT64_SCHEMA) .field("itemid".toUpperCase(), org.apache.kafka.connect.data.Schema.OPTIONAL_STRING_SCHEMA) .field("orderunits".toUpperCase(), org.apache.kafka.connect.data.Schema.OPTIONAL_FLOAT64_SCHEMA) .build(); final KsqlJsonDeserializer ksqlJsonDeserializer = new KsqlJsonDeserializer( newOrderSchema, false, recordLogger); final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", jsonBytes); assertThat(genericRow.getColumns().size(), equalTo(4)); assertThat(genericRow.getColumns().get(0), equalTo(1511897796092L)); assertThat(genericRow.getColumns().get(1), equalTo(1L)); assertThat(genericRow.getColumns().get(2), equalTo("Item_1")); assertThat(genericRow.getColumns().get(3), equalTo(10.0)); }
@Before public void before() { orderSchema = SchemaBuilder.struct() .field("ordertime".toUpperCase(), org.apache.kafka.connect.data.Schema.OPTIONAL_INT64_SCHEMA) .field("orderid".toUpperCase(), org.apache.kafka.connect.data.Schema.OPTIONAL_INT64_SCHEMA) .field("itemid".toUpperCase(), org.apache.kafka.connect.data.Schema.OPTIONAL_STRING_SCHEMA) .field("orderunits".toUpperCase(), org.apache.kafka.connect.data.Schema.OPTIONAL_FLOAT64_SCHEMA) .field("arraycol".toUpperCase(), SchemaBuilder.array(org.apache.kafka.connect.data.Schema.OPTIONAL_FLOAT64_SCHEMA).optional().build()) .field("mapcol".toUpperCase(), SchemaBuilder.map(org.apache.kafka.connect.data.Schema.OPTIONAL_STRING_SCHEMA, org.apache.kafka.connect.data.Schema.OPTIONAL_FLOAT64_SCHEMA).optional().build()) .build(); ksqlJsonDeserializer = new KsqlJsonDeserializer( orderSchema, false, recordLogger); }
@Test public void shouldDeserializeJsonCorrectly() throws JsonProcessingException { final Map<String, Object> orderRow = new HashMap<>(); orderRow.put("ordertime", 1511897796092L); orderRow.put("@orderid", 1L); orderRow.put("itemid", "Item_1"); orderRow.put("orderunits", 10.0); orderRow.put("arraycol", new Double[]{10.0, 20.0}); orderRow.put("mapcol", Collections.singletonMap("key1", 10.0)); final byte[] jsonBytes = objectMapper.writeValueAsBytes(orderRow); final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", jsonBytes); assertThat(genericRow.getColumns().size(), equalTo(6)); assertThat(genericRow.getColumns().get(0), equalTo(1511897796092L)); assertThat(genericRow.getColumns().get(1), equalTo(1L)); assertThat(genericRow.getColumns().get(2), equalTo("Item_1")); assertThat(genericRow.getColumns().get(3), equalTo(10.0)); }
@Test public void shouldDeserializeEvenWithMissingFields() throws JsonProcessingException { final Map<String, Object> orderRow = new HashMap<>(); orderRow.put("ordertime", 1511897796092L); orderRow.put("@orderid", 1L); orderRow.put("itemid", "Item_1"); orderRow.put("orderunits", 10.0); final byte[] jsonBytes = objectMapper.writeValueAsBytes(orderRow); final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", jsonBytes); assertThat(genericRow.getColumns().size(), equalTo(6)); assertThat(genericRow.getColumns().get(0), equalTo(1511897796092L)); assertThat(genericRow.getColumns().get(1), equalTo(1L)); assertThat(genericRow.getColumns().get(2), equalTo("Item_1")); assertThat(genericRow.getColumns().get(3), equalTo(10.0)); assertThat(genericRow.getColumns().get(4), is(nullValue())); assertThat(genericRow.getColumns().get(5), is(nullValue())); }
private List<?> enforceFieldTypeForArray(final Schema fieldSchema, final List<?> arrayList) { final List<Object> array = new ArrayList<>(arrayList.size()); for (final Object item : arrayList) { array.add(enforceFieldType(fieldSchema.valueSchema(), item)); } return array; }
@SuppressWarnings("unchecked") private GenericRow getGenericRow(final byte[] rowJsonBytes) { final SchemaAndValue schemaAndValue = jsonConverter.toConnectData("topic", rowJsonBytes); final Map<String, Object> valueMap = (Map) schemaAndValue.value(); if (valueMap == null) { return null; } final Map<String, String> caseInsensitiveFieldNameMap = getCaseInsensitiveFieldNameMap(valueMap, true); final List<Object> columns = new ArrayList(schema.fields().size()); for (final Field field : schema.fields()) { final Object columnVal = valueMap.get(caseInsensitiveFieldNameMap.get(field.name())); columns.add(enforceFieldType(field.schema(), columnVal)); } return new GenericRow(columns); }
@Test public void shouldLogDeserializationErrors() { // When: Throwable cause = null; final byte[] data = "{foo".getBytes(StandardCharsets.UTF_8); try { ksqlJsonDeserializer.deserialize("", data); fail("deserialize should have thrown"); } catch (final SerializationException e) { cause = e.getCause(); } // Then: SerdeTestUtils.shouldLogError( recordLogger, ProcessingLogMessageFactory.deserializationErrorMsg(cause, Optional.ofNullable(data)).get()); } }