in connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java [578:691]
private static JsonNode convertToJson(Schema schema, Object logicalValue) {
if (logicalValue == null) {
if (schema == null) // Any schema is valid and we don't have a default, so treat this as an optional schema
return null;
if (schema.defaultValue() != null)
return convertToJson(schema, schema.defaultValue());
if (schema.isOptional())
return JsonNodeFactory.instance.nullNode();
throw new DataException("Conversion error: null value for field that is required and has no default value");
}
Object value = logicalValue;
if (schema != null && schema.name() != null) {
LogicalTypeConverter logicalConverter = TO_JSON_LOGICAL_CONVERTERS.get(schema.name());
if (logicalConverter != null)
value = logicalConverter.convert(schema, logicalValue);
}
try {
final Schema.Type schemaType;
if (schema == null) {
schemaType = ConnectSchema.schemaType(value.getClass());
if (schemaType == null)
throw new DataException("Java class " + value.getClass() + " does not have corresponding schema type.");
} else {
schemaType = schema.type();
}
switch (schemaType) {
case INT8:
return JsonNodeFactory.instance.numberNode((Byte) value);
case INT16:
return JsonNodeFactory.instance.numberNode((Short) value);
case INT32:
return JsonNodeFactory.instance.numberNode((Integer) value);
case INT64:
return JsonNodeFactory.instance.numberNode((Long) value);
case FLOAT32:
return JsonNodeFactory.instance.numberNode((Float) value);
case FLOAT64:
return JsonNodeFactory.instance.numberNode((Double) value);
case BOOLEAN:
return JsonNodeFactory.instance.booleanNode((Boolean) value);
case STRING:
CharSequence charSeq = (CharSequence) value;
return JsonNodeFactory.instance.textNode(charSeq.toString());
case BYTES:
if (value instanceof byte[])
return JsonNodeFactory.instance.binaryNode((byte[]) value);
else if (value instanceof ByteBuffer)
return JsonNodeFactory.instance.binaryNode(((ByteBuffer) value).array());
else
throw new DataException("Invalid type for bytes type: " + value.getClass());
case ARRAY: {
Collection collection = (Collection) value;
ArrayNode list = JsonNodeFactory.instance.arrayNode();
for (Object elem : collection) {
Schema valueSchema = schema == null ? null : schema.valueSchema();
JsonNode fieldValue = convertToJson(valueSchema, elem);
list.add(fieldValue);
}
return list;
}
case MAP: {
Map<?, ?> map = (Map<?, ?>) value;
// If true, using string keys and JSON object; if false, using non-string keys and Array-encoding
boolean objectMode;
if (schema == null) {
objectMode = true;
for (Map.Entry<?, ?> entry : map.entrySet()) {
if (!(entry.getKey() instanceof String)) {
objectMode = false;
break;
}
}
} else {
objectMode = schema.keySchema().type() == Schema.Type.STRING;
}
ObjectNode obj = null;
ArrayNode list = null;
if (objectMode)
obj = JsonNodeFactory.instance.objectNode();
else
list = JsonNodeFactory.instance.arrayNode();
for (Map.Entry<?, ?> entry : map.entrySet()) {
Schema keySchema = schema == null ? null : schema.keySchema();
Schema valueSchema = schema == null ? null : schema.valueSchema();
JsonNode mapKey = convertToJson(keySchema, entry.getKey());
JsonNode mapValue = convertToJson(valueSchema, entry.getValue());
if (objectMode)
obj.set(mapKey.asText(), mapValue);
else
list.add(JsonNodeFactory.instance.arrayNode().add(mapKey).add(mapValue));
}
return objectMode ? obj : list;
}
case STRUCT: {
Struct struct = (Struct) value;
if (struct.schema() != schema)
throw new DataException("Mismatching schema.");
ObjectNode obj = JsonNodeFactory.instance.objectNode();
for (Field field : schema.fields()) {
obj.set(field.name(), convertToJson(field.schema(), struct.get(field)));
}
return obj;
}
}
throw new DataException("Couldn't convert " + value + " to JSON.");
} catch (ClassCastException e) {
String schemaTypeStr = (schema != null) ? schema.type().toString() : "unknown schema";
throw new DataException("Invalid type for " + schemaTypeStr + ": " + value.getClass());
}
}