public ObjectNode asJsonSchema()

in connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java [389:467]


    public 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 = JSON_NODE_FACTORY.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 = JSON_NODE_FACTORY.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 = JSON_NODE_FACTORY.objectNode().put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.STRUCT_TYPE_NAME);
                ArrayNode fields = JSON_NODE_FACTORY.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 = JSON_NODE_FACTORY.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;
    }