private static Object checkOptionalAndDefault(Schema schema) { if (schema.defaultValue() != null) return schema.defaultValue(); if (schema.isOptional()) return null; throw new DataException("Invalid null value for required field"); }
/** * Convert this object, in the org.apache.kafka.connect.data format, into a JSON object, returning * both the schema and the converted object. */ 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) { throw new DataException("Invalid type for " + schema.type() + ": " + value.getClass()); } }
private ObjectNode asJsonSchema(Schema schema) { if (schema == null) return null; ObjectNode cached = fromConnectSchemaCache.get(schema); if (cached != null) return cached; final ObjectNode jsonSchema; switch (schema.type()) { case BOOLEAN: jsonSchema = JsonSchema.BOOLEAN_SCHEMA.deepCopy(); break; case BYTES: jsonSchema = JsonSchema.BYTES_SCHEMA.deepCopy(); break; case FLOAT64: jsonSchema = JsonSchema.DOUBLE_SCHEMA.deepCopy(); break; case FLOAT32: jsonSchema = JsonSchema.FLOAT_SCHEMA.deepCopy(); break; case INT8: jsonSchema = JsonSchema.INT8_SCHEMA.deepCopy(); break; case INT16: jsonSchema = JsonSchema.INT16_SCHEMA.deepCopy(); break; case INT32: jsonSchema = JsonSchema.INT32_SCHEMA.deepCopy(); break; case INT64: jsonSchema = JsonSchema.INT64_SCHEMA.deepCopy(); break; case STRING: jsonSchema = JsonSchema.STRING_SCHEMA.deepCopy(); break; case ARRAY: jsonSchema = JsonNodeFactory.instance .objectNode() .put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.ARRAY_TYPE_NAME); jsonSchema.set(JsonSchema.ARRAY_ITEMS_FIELD_NAME, asJsonSchema(schema.valueSchema())); break; case MAP: jsonSchema = JsonNodeFactory.instance .objectNode() .put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.MAP_TYPE_NAME); jsonSchema.set(JsonSchema.MAP_KEY_FIELD_NAME, asJsonSchema(schema.keySchema())); jsonSchema.set(JsonSchema.MAP_VALUE_FIELD_NAME, asJsonSchema(schema.valueSchema())); break; case STRUCT: jsonSchema = JsonNodeFactory.instance .objectNode() .put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.STRUCT_TYPE_NAME); ArrayNode fields = JsonNodeFactory.instance.arrayNode(); for (Field field : schema.fields()) { ObjectNode fieldJsonSchema = asJsonSchema(field.schema()).deepCopy(); fieldJsonSchema.put(JsonSchema.STRUCT_FIELD_NAME_FIELD_NAME, field.name()); fields.add(fieldJsonSchema); } jsonSchema.set(JsonSchema.STRUCT_FIELDS_FIELD_NAME, fields); break; default: throw new DataException("Couldn't translate unsupported schema type " + schema + "."); } jsonSchema.put(JsonSchema.SCHEMA_OPTIONAL_FIELD_NAME, schema.isOptional()); if (schema.name() != null) jsonSchema.put(JsonSchema.SCHEMA_NAME_FIELD_NAME, schema.name()); if (schema.version() != null) jsonSchema.put(JsonSchema.SCHEMA_VERSION_FIELD_NAME, schema.version()); if (schema.doc() != null) jsonSchema.put(JsonSchema.SCHEMA_DOC_FIELD_NAME, schema.doc()); if (schema.parameters() != null) { ObjectNode jsonSchemaParams = JsonNodeFactory.instance.objectNode(); for (Map.Entry<String, String> prop : schema.parameters().entrySet()) jsonSchemaParams.put(prop.getKey(), prop.getValue()); jsonSchema.set(JsonSchema.SCHEMA_PARAMETERS_FIELD_NAME, jsonSchemaParams); } if (schema.defaultValue() != null) jsonSchema.set( JsonSchema.SCHEMA_DEFAULT_FIELD_NAME, convertToJson(schema, schema.defaultValue())); fromConnectSchemaCache.put(schema, jsonSchema); return jsonSchema; }