Add km module kafka

This commit is contained in:
leewei
2023-02-14 14:57:39 +08:00
parent 229140f067
commit 469baad65b
4310 changed files with 736354 additions and 46204 deletions

1
connect/json/.gitignore vendored Normal file
View File

@@ -0,0 +1 @@
/bin/

View File

@@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.json;
/**
* Represents the valid {@link org.apache.kafka.connect.data.Decimal} serialization formats
* in a {@link JsonConverter}.
*/
public enum DecimalFormat {
/**
* Serializes the JSON Decimal as a base-64 string. For example, serializing the value
* `10.2345` with the BASE64 setting will result in `"D3J5"`.
*/
BASE64,
/**
* Serializes the JSON Decimal as a JSON number. For example, serializing the value
* `10.2345` with the NUMERIC setting will result in `10.2345`.
*/
NUMERIC
}

View File

@@ -0,0 +1,783 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.json;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.kafka.common.cache.Cache;
import org.apache.kafka.common.cache.LRUCache;
import org.apache.kafka.common.cache.SynchronizedCache;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.ConnectSchema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.data.Time;
import org.apache.kafka.connect.data.Decimal;
import org.apache.kafka.connect.data.Date;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.storage.Converter;
import org.apache.kafka.connect.storage.ConverterType;
import org.apache.kafka.connect.storage.HeaderConverter;
import org.apache.kafka.connect.storage.StringConverterConfig;
import java.io.IOException;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import static org.apache.kafka.common.utils.Utils.mkSet;
/**
* Implementation of Converter that uses JSON to store schemas and objects. By default this converter will serialize Connect keys, values,
* and headers with schemas, although this can be disabled with {@link JsonConverterConfig#SCHEMAS_ENABLE_CONFIG schemas.enable}
* configuration option.
*
* This implementation currently does nothing with the topic names or header names.
*/
public class JsonConverter implements Converter, HeaderConverter {
private static final Map<Schema.Type, JsonToConnectTypeConverter> TO_CONNECT_CONVERTERS = new EnumMap<>(Schema.Type.class);
static {
TO_CONNECT_CONVERTERS.put(Schema.Type.BOOLEAN, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
return value.booleanValue();
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.INT8, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
return (byte) value.intValue();
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.INT16, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
return (short) value.intValue();
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.INT32, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
return value.intValue();
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.INT64, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
return value.longValue();
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT32, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
return value.floatValue();
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT64, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
return value.doubleValue();
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.BYTES, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
try {
return value.binaryValue();
} catch (IOException e) {
throw new DataException("Invalid bytes field", e);
}
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.STRING, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
return value.textValue();
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.ARRAY, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
Schema elemSchema = schema == null ? null : schema.valueSchema();
ArrayList<Object> result = new ArrayList<>();
for (JsonNode elem : value) {
result.add(convertToConnect(elemSchema, elem));
}
return result;
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.MAP, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
Schema keySchema = schema == null ? null : schema.keySchema();
Schema valueSchema = schema == null ? null : schema.valueSchema();
// If the map uses strings for keys, it should be encoded in the natural JSON format. If it uses other
// primitive types or a complex type as a key, it will be encoded as a list of pairs. If we don't have a
// schema, we default to encoding in a Map.
Map<Object, Object> result = new HashMap<>();
if (schema == null || keySchema.type() == Schema.Type.STRING) {
if (!value.isObject())
throw new DataException("Maps with string fields should be encoded as JSON objects, but found " + value.getNodeType());
Iterator<Map.Entry<String, JsonNode>> fieldIt = value.fields();
while (fieldIt.hasNext()) {
Map.Entry<String, JsonNode> entry = fieldIt.next();
result.put(entry.getKey(), convertToConnect(valueSchema, entry.getValue()));
}
} else {
if (!value.isArray())
throw new DataException("Maps with non-string fields should be encoded as JSON array of tuples, but found " + value.getNodeType());
for (JsonNode entry : value) {
if (!entry.isArray())
throw new DataException("Found invalid map entry instead of array tuple: " + entry.getNodeType());
if (entry.size() != 2)
throw new DataException("Found invalid map entry, expected length 2 but found :" + entry.size());
result.put(convertToConnect(keySchema, entry.get(0)),
convertToConnect(valueSchema, entry.get(1)));
}
}
return result;
}
});
TO_CONNECT_CONVERTERS.put(Schema.Type.STRUCT, new JsonToConnectTypeConverter() {
@Override
public Object convert(Schema schema, JsonNode value) {
if (!value.isObject())
throw new DataException("Structs should be encoded as JSON objects, but found " + value.getNodeType());
// We only have ISchema here but need Schema, so we need to materialize the actual schema. Using ISchema
// avoids having to materialize the schema for non-Struct types but it cannot be avoided for Structs since
// they require a schema to be provided at construction. However, the schema is only a SchemaBuilder during
// translation of schemas to JSON; during the more common translation of data to JSON, the call to schema.schema()
// just returns the schema Object and has no overhead.
Struct result = new Struct(schema.schema());
for (Field field : schema.fields())
result.put(field, convertToConnect(field.schema(), value.get(field.name())));
return result;
}
});
}
// Convert values in Kafka Connect form into/from their logical types. These logical converters are discovered by logical type
// names specified in the field
private static final HashMap<String, LogicalTypeConverter> LOGICAL_CONVERTERS = new HashMap<>();
private static final JsonNodeFactory JSON_NODE_FACTORY = JsonNodeFactory.withExactBigDecimals(true);
static {
LOGICAL_CONVERTERS.put(Decimal.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public JsonNode toJson(final Schema schema, final Object value, final JsonConverterConfig config) {
if (!(value instanceof BigDecimal))
throw new DataException("Invalid type for Decimal, expected BigDecimal but was " + value.getClass());
final BigDecimal decimal = (BigDecimal) value;
switch (config.decimalFormat()) {
case NUMERIC:
return JSON_NODE_FACTORY.numberNode(decimal);
case BASE64:
return JSON_NODE_FACTORY.binaryNode(Decimal.fromLogical(schema, decimal));
default:
throw new DataException("Unexpected " + JsonConverterConfig.DECIMAL_FORMAT_CONFIG + ": " + config.decimalFormat());
}
}
@Override
public Object toConnect(final Schema schema, final JsonNode value) {
if (value.isNumber()) return value.decimalValue();
if (value.isBinary() || value.isTextual()) {
try {
return Decimal.toLogical(schema, value.binaryValue());
} catch (Exception e) {
throw new DataException("Invalid bytes for Decimal field", e);
}
}
throw new DataException("Invalid type for Decimal, underlying representation should be numeric or bytes but was " + value.getNodeType());
}
});
LOGICAL_CONVERTERS.put(Date.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public JsonNode toJson(final Schema schema, final Object value, final JsonConverterConfig config) {
if (!(value instanceof java.util.Date))
throw new DataException("Invalid type for Date, expected Date but was " + value.getClass());
return JSON_NODE_FACTORY.numberNode(Date.fromLogical(schema, (java.util.Date) value));
}
@Override
public Object toConnect(final Schema schema, final JsonNode value) {
if (!(value.isInt()))
throw new DataException("Invalid type for Date, underlying representation should be integer but was " + value.getNodeType());
return Date.toLogical(schema, value.intValue());
}
});
LOGICAL_CONVERTERS.put(Time.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public JsonNode toJson(final Schema schema, final Object value, final JsonConverterConfig config) {
if (!(value instanceof java.util.Date))
throw new DataException("Invalid type for Time, expected Date but was " + value.getClass());
return JSON_NODE_FACTORY.numberNode(Time.fromLogical(schema, (java.util.Date) value));
}
@Override
public Object toConnect(final Schema schema, final JsonNode value) {
if (!(value.isInt()))
throw new DataException("Invalid type for Time, underlying representation should be integer but was " + value.getNodeType());
return Time.toLogical(schema, value.intValue());
}
});
LOGICAL_CONVERTERS.put(Timestamp.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public JsonNode toJson(final Schema schema, final Object value, final JsonConverterConfig config) {
if (!(value instanceof java.util.Date))
throw new DataException("Invalid type for Timestamp, expected Date but was " + value.getClass());
return JSON_NODE_FACTORY.numberNode(Timestamp.fromLogical(schema, (java.util.Date) value));
}
@Override
public Object toConnect(final Schema schema, final JsonNode value) {
if (!(value.isIntegralNumber()))
throw new DataException("Invalid type for Timestamp, underlying representation should be integral but was " + value.getNodeType());
return Timestamp.toLogical(schema, value.longValue());
}
});
}
private JsonConverterConfig config;
private Cache<Schema, ObjectNode> fromConnectSchemaCache;
private Cache<JsonNode, Schema> toConnectSchemaCache;
private final JsonSerializer serializer;
private final JsonDeserializer deserializer;
public JsonConverter() {
serializer = new JsonSerializer(
mkSet(),
JSON_NODE_FACTORY
);
deserializer = new JsonDeserializer(
mkSet(
// this ensures that the JsonDeserializer maintains full precision on
// floating point numbers that cannot fit into float64
DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS
),
JSON_NODE_FACTORY
);
}
@Override
public ConfigDef config() {
return JsonConverterConfig.configDef();
}
@Override
public void configure(Map<String, ?> configs) {
config = new JsonConverterConfig(configs);
serializer.configure(configs, config.type() == ConverterType.KEY);
deserializer.configure(configs, config.type() == ConverterType.KEY);
fromConnectSchemaCache = new SynchronizedCache<>(new LRUCache<>(config.schemaCacheSize()));
toConnectSchemaCache = new SynchronizedCache<>(new LRUCache<>(config.schemaCacheSize()));
}
@Override
public void configure(Map<String, ?> configs, boolean isKey) {
Map<String, Object> conf = new HashMap<>(configs);
conf.put(StringConverterConfig.TYPE_CONFIG, isKey ? ConverterType.KEY.getName() : ConverterType.VALUE.getName());
configure(conf);
}
@Override
public void close() {
// do nothing
}
@Override
public byte[] fromConnectHeader(String topic, String headerKey, Schema schema, Object value) {
return fromConnectData(topic, schema, value);
}
@Override
public SchemaAndValue toConnectHeader(String topic, String headerKey, byte[] value) {
return toConnectData(topic, value);
}
@Override
public byte[] fromConnectData(String topic, Schema schema, Object value) {
if (schema == null && value == null) {
return null;
}
JsonNode jsonValue = config.schemasEnabled() ? convertToJsonWithEnvelope(schema, value) : convertToJsonWithoutEnvelope(schema, value);
try {
return serializer.serialize(topic, jsonValue);
} catch (SerializationException e) {
throw new DataException("Converting Kafka Connect data to byte[] failed due to serialization error: ", e);
}
}
@Override
public SchemaAndValue toConnectData(String topic, byte[] value) {
JsonNode jsonValue;
// This handles a tombstone message
if (value == null) {
return SchemaAndValue.NULL;
}
try {
jsonValue = deserializer.deserialize(topic, value);
} catch (SerializationException e) {
throw new DataException("Converting byte[] to Kafka Connect data failed due to serialization error: ", e);
}
if (config.schemasEnabled() && (!jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !jsonValue.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)))
throw new DataException("JsonConverter with schemas.enable requires \"schema\" and \"payload\" fields and may not contain additional fields." +
" If you are trying to deserialize plain JSON data, set schemas.enable=false in your converter configuration.");
// The deserialized data should either be an envelope object containing the schema and the payload or the schema
// was stripped during serialization and we need to fill in an all-encompassing schema.
if (!config.schemasEnabled()) {
ObjectNode envelope = JSON_NODE_FACTORY.objectNode();
envelope.set(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME, null);
envelope.set(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME, jsonValue);
jsonValue = envelope;
}
Schema schema = asConnectSchema(jsonValue.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
return new SchemaAndValue(
schema,
convertToConnect(schema, jsonValue.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME))
);
}
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;
}
public Schema asConnectSchema(JsonNode jsonSchema) {
if (jsonSchema.isNull())
return null;
Schema cached = toConnectSchemaCache.get(jsonSchema);
if (cached != null)
return cached;
JsonNode schemaTypeNode = jsonSchema.get(JsonSchema.SCHEMA_TYPE_FIELD_NAME);
if (schemaTypeNode == null || !schemaTypeNode.isTextual())
throw new DataException("Schema must contain 'type' field");
final SchemaBuilder builder;
switch (schemaTypeNode.textValue()) {
case JsonSchema.BOOLEAN_TYPE_NAME:
builder = SchemaBuilder.bool();
break;
case JsonSchema.INT8_TYPE_NAME:
builder = SchemaBuilder.int8();
break;
case JsonSchema.INT16_TYPE_NAME:
builder = SchemaBuilder.int16();
break;
case JsonSchema.INT32_TYPE_NAME:
builder = SchemaBuilder.int32();
break;
case JsonSchema.INT64_TYPE_NAME:
builder = SchemaBuilder.int64();
break;
case JsonSchema.FLOAT_TYPE_NAME:
builder = SchemaBuilder.float32();
break;
case JsonSchema.DOUBLE_TYPE_NAME:
builder = SchemaBuilder.float64();
break;
case JsonSchema.BYTES_TYPE_NAME:
builder = SchemaBuilder.bytes();
break;
case JsonSchema.STRING_TYPE_NAME:
builder = SchemaBuilder.string();
break;
case JsonSchema.ARRAY_TYPE_NAME:
JsonNode elemSchema = jsonSchema.get(JsonSchema.ARRAY_ITEMS_FIELD_NAME);
if (elemSchema == null || elemSchema.isNull())
throw new DataException("Array schema did not specify the element type");
builder = SchemaBuilder.array(asConnectSchema(elemSchema));
break;
case JsonSchema.MAP_TYPE_NAME:
JsonNode keySchema = jsonSchema.get(JsonSchema.MAP_KEY_FIELD_NAME);
if (keySchema == null)
throw new DataException("Map schema did not specify the key type");
JsonNode valueSchema = jsonSchema.get(JsonSchema.MAP_VALUE_FIELD_NAME);
if (valueSchema == null)
throw new DataException("Map schema did not specify the value type");
builder = SchemaBuilder.map(asConnectSchema(keySchema), asConnectSchema(valueSchema));
break;
case JsonSchema.STRUCT_TYPE_NAME:
builder = SchemaBuilder.struct();
JsonNode fields = jsonSchema.get(JsonSchema.STRUCT_FIELDS_FIELD_NAME);
if (fields == null || !fields.isArray())
throw new DataException("Struct schema's \"fields\" argument is not an array.");
for (JsonNode field : fields) {
JsonNode jsonFieldName = field.get(JsonSchema.STRUCT_FIELD_NAME_FIELD_NAME);
if (jsonFieldName == null || !jsonFieldName.isTextual())
throw new DataException("Struct schema's field name not specified properly");
builder.field(jsonFieldName.asText(), asConnectSchema(field));
}
break;
default:
throw new DataException("Unknown schema type: " + schemaTypeNode.textValue());
}
JsonNode schemaOptionalNode = jsonSchema.get(JsonSchema.SCHEMA_OPTIONAL_FIELD_NAME);
if (schemaOptionalNode != null && schemaOptionalNode.isBoolean() && schemaOptionalNode.booleanValue())
builder.optional();
else
builder.required();
JsonNode schemaNameNode = jsonSchema.get(JsonSchema.SCHEMA_NAME_FIELD_NAME);
if (schemaNameNode != null && schemaNameNode.isTextual())
builder.name(schemaNameNode.textValue());
JsonNode schemaVersionNode = jsonSchema.get(JsonSchema.SCHEMA_VERSION_FIELD_NAME);
if (schemaVersionNode != null && schemaVersionNode.isIntegralNumber()) {
builder.version(schemaVersionNode.intValue());
}
JsonNode schemaDocNode = jsonSchema.get(JsonSchema.SCHEMA_DOC_FIELD_NAME);
if (schemaDocNode != null && schemaDocNode.isTextual())
builder.doc(schemaDocNode.textValue());
JsonNode schemaParamsNode = jsonSchema.get(JsonSchema.SCHEMA_PARAMETERS_FIELD_NAME);
if (schemaParamsNode != null && schemaParamsNode.isObject()) {
Iterator<Map.Entry<String, JsonNode>> paramsIt = schemaParamsNode.fields();
while (paramsIt.hasNext()) {
Map.Entry<String, JsonNode> entry = paramsIt.next();
JsonNode paramValue = entry.getValue();
if (!paramValue.isTextual())
throw new DataException("Schema parameters must have string values.");
builder.parameter(entry.getKey(), paramValue.textValue());
}
}
JsonNode schemaDefaultNode = jsonSchema.get(JsonSchema.SCHEMA_DEFAULT_FIELD_NAME);
if (schemaDefaultNode != null)
builder.defaultValue(convertToConnect(builder, schemaDefaultNode));
Schema result = builder.build();
toConnectSchemaCache.put(jsonSchema, result);
return result;
}
/**
* Convert this object, in org.apache.kafka.connect.data format, into a JSON object with an envelope object
* containing schema and payload fields.
* @param schema the schema for the data
* @param value the value
* @return JsonNode-encoded version
*/
private JsonNode convertToJsonWithEnvelope(Schema schema, Object value) {
return new JsonSchema.Envelope(asJsonSchema(schema), convertToJson(schema, value)).toJsonNode();
}
private JsonNode convertToJsonWithoutEnvelope(Schema schema, Object value) {
return convertToJson(schema, value);
}
/**
* Convert this object, in the org.apache.kafka.connect.data format, into a JSON object, returning both the schema
* and the converted object.
*/
private JsonNode convertToJson(Schema schema, Object value) {
if (value == 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 JSON_NODE_FACTORY.nullNode();
throw new DataException("Conversion error: null value for field that is required and has no default value");
}
if (schema != null && schema.name() != null) {
LogicalTypeConverter logicalConverter = LOGICAL_CONVERTERS.get(schema.name());
if (logicalConverter != null)
return logicalConverter.toJson(schema, value, config);
}
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 JSON_NODE_FACTORY.numberNode((Byte) value);
case INT16:
return JSON_NODE_FACTORY.numberNode((Short) value);
case INT32:
return JSON_NODE_FACTORY.numberNode((Integer) value);
case INT64:
return JSON_NODE_FACTORY.numberNode((Long) value);
case FLOAT32:
return JSON_NODE_FACTORY.numberNode((Float) value);
case FLOAT64:
return JSON_NODE_FACTORY.numberNode((Double) value);
case BOOLEAN:
return JSON_NODE_FACTORY.booleanNode((Boolean) value);
case STRING:
CharSequence charSeq = (CharSequence) value;
return JSON_NODE_FACTORY.textNode(charSeq.toString());
case BYTES:
if (value instanceof byte[])
return JSON_NODE_FACTORY.binaryNode((byte[]) value);
else if (value instanceof ByteBuffer)
return JSON_NODE_FACTORY.binaryNode(((ByteBuffer) value).array());
else
throw new DataException("Invalid type for bytes type: " + value.getClass());
case ARRAY: {
Collection collection = (Collection) value;
ArrayNode list = JSON_NODE_FACTORY.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 = JSON_NODE_FACTORY.objectNode();
else
list = JSON_NODE_FACTORY.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(JSON_NODE_FACTORY.arrayNode().add(mapKey).add(mapValue));
}
return objectMode ? obj : list;
}
case STRUCT: {
Struct struct = (Struct) value;
if (!struct.schema().equals(schema))
throw new DataException("Mismatching schema.");
ObjectNode obj = JSON_NODE_FACTORY.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());
}
}
private static Object convertToConnect(Schema schema, JsonNode jsonValue) {
final Schema.Type schemaType;
if (schema != null) {
schemaType = schema.type();
if (jsonValue == null || jsonValue.isNull()) {
if (schema.defaultValue() != null)
return schema.defaultValue(); // any logical type conversions should already have been applied
if (schema.isOptional())
return null;
throw new DataException("Invalid null value for required " + schemaType + " field");
}
} else {
switch (jsonValue.getNodeType()) {
case NULL:
// Special case. With no schema
return null;
case BOOLEAN:
schemaType = Schema.Type.BOOLEAN;
break;
case NUMBER:
if (jsonValue.isIntegralNumber())
schemaType = Schema.Type.INT64;
else
schemaType = Schema.Type.FLOAT64;
break;
case ARRAY:
schemaType = Schema.Type.ARRAY;
break;
case OBJECT:
schemaType = Schema.Type.MAP;
break;
case STRING:
schemaType = Schema.Type.STRING;
break;
case BINARY:
case MISSING:
case POJO:
default:
schemaType = null;
break;
}
}
final JsonToConnectTypeConverter typeConverter = TO_CONNECT_CONVERTERS.get(schemaType);
if (typeConverter == null)
throw new DataException("Unknown schema type: " + schemaType);
if (schema != null && schema.name() != null) {
LogicalTypeConverter logicalConverter = LOGICAL_CONVERTERS.get(schema.name());
if (logicalConverter != null)
return logicalConverter.toConnect(schema, jsonValue);
}
return typeConverter.convert(schema, jsonValue);
}
private interface JsonToConnectTypeConverter {
Object convert(Schema schema, JsonNode value);
}
private interface LogicalTypeConverter {
JsonNode toJson(Schema schema, Object value, JsonConverterConfig config);
Object toConnect(Schema schema, JsonNode value);
}
}

View File

@@ -0,0 +1,114 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.json;
import java.util.Locale;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigDef.Width;
import org.apache.kafka.connect.storage.ConverterConfig;
import java.util.Map;
/**
* Configuration options for {@link JsonConverter} instances.
*/
public class JsonConverterConfig extends ConverterConfig {
public static final String SCHEMAS_ENABLE_CONFIG = "schemas.enable";
public static final boolean SCHEMAS_ENABLE_DEFAULT = true;
private static final String SCHEMAS_ENABLE_DOC = "Include schemas within each of the serialized values and keys.";
private static final String SCHEMAS_ENABLE_DISPLAY = "Enable Schemas";
public static final String SCHEMAS_CACHE_SIZE_CONFIG = "schemas.cache.size";
public static final int SCHEMAS_CACHE_SIZE_DEFAULT = 1000;
private static final String SCHEMAS_CACHE_SIZE_DOC = "The maximum number of schemas that can be cached in this converter instance.";
private static final String SCHEMAS_CACHE_SIZE_DISPLAY = "Schema Cache Size";
public static final String DECIMAL_FORMAT_CONFIG = "decimal.format";
public static final String DECIMAL_FORMAT_DEFAULT = DecimalFormat.BASE64.name();
private static final String DECIMAL_FORMAT_DOC = "Controls which format this converter will serialize decimals in."
+ " This value is case insensitive and can be either 'BASE64' (default) or 'NUMERIC'";
private static final String DECIMAL_FORMAT_DISPLAY = "Decimal Format";
private final static ConfigDef CONFIG;
static {
String group = "Schemas";
int orderInGroup = 0;
CONFIG = ConverterConfig.newConfigDef();
CONFIG.define(SCHEMAS_ENABLE_CONFIG, Type.BOOLEAN, SCHEMAS_ENABLE_DEFAULT, Importance.HIGH, SCHEMAS_ENABLE_DOC, group,
orderInGroup++, Width.MEDIUM, SCHEMAS_ENABLE_DISPLAY);
CONFIG.define(SCHEMAS_CACHE_SIZE_CONFIG, Type.INT, SCHEMAS_CACHE_SIZE_DEFAULT, Importance.HIGH, SCHEMAS_CACHE_SIZE_DOC, group,
orderInGroup++, Width.MEDIUM, SCHEMAS_CACHE_SIZE_DISPLAY);
group = "Serialization";
orderInGroup = 0;
CONFIG.define(
DECIMAL_FORMAT_CONFIG, Type.STRING, DECIMAL_FORMAT_DEFAULT,
ConfigDef.CaseInsensitiveValidString.in(
DecimalFormat.BASE64.name(),
DecimalFormat.NUMERIC.name()),
Importance.LOW, DECIMAL_FORMAT_DOC, group, orderInGroup++,
Width.MEDIUM, DECIMAL_FORMAT_DISPLAY);
}
public static ConfigDef configDef() {
return CONFIG;
}
// cached config values
private final boolean schemasEnabled;
private final int schemaCacheSize;
private final DecimalFormat decimalFormat;
public JsonConverterConfig(Map<String, ?> props) {
super(CONFIG, props);
this.schemasEnabled = getBoolean(SCHEMAS_ENABLE_CONFIG);
this.schemaCacheSize = getInt(SCHEMAS_CACHE_SIZE_CONFIG);
this.decimalFormat = DecimalFormat.valueOf(getString(DECIMAL_FORMAT_CONFIG).toUpperCase(Locale.ROOT));
}
/**
* Return whether schemas are enabled.
*
* @return true if enabled, or false otherwise
*/
public boolean schemasEnabled() {
return schemasEnabled;
}
/**
* Get the cache size.
*
* @return the cache size
*/
public int schemaCacheSize() {
return schemaCacheSize;
}
/**
* Get the serialization format for decimal types.
*
* @return the decimal serialization format
*/
public DecimalFormat decimalFormat() {
return decimalFormat;
}
}

View File

@@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.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.JsonNodeFactory;
import java.util.Collections;
import java.util.Set;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.serialization.Deserializer;
/**
* JSON deserializer for Jackson's JsonNode tree model. Using the tree model allows it to work with arbitrarily
* structured data without having associated Java classes. This deserializer also supports Connect schemas.
*/
public class JsonDeserializer implements Deserializer<JsonNode> {
private final ObjectMapper objectMapper = new ObjectMapper();
/**
* Default constructor needed by Kafka
*/
public JsonDeserializer() {
this(Collections.emptySet(), JsonNodeFactory.withExactBigDecimals(true));
}
/**
* A constructor that additionally specifies some {@link DeserializationFeature}
* for the deserializer
*
* @param deserializationFeatures the specified deserialization features
* @param jsonNodeFactory the json node factory to use.
*/
JsonDeserializer(
final Set<DeserializationFeature> deserializationFeatures,
final JsonNodeFactory jsonNodeFactory
) {
deserializationFeatures.forEach(objectMapper::enable);
objectMapper.setNodeFactory(jsonNodeFactory);
}
@Override
public JsonNode deserialize(String topic, byte[] bytes) {
if (bytes == null)
return null;
JsonNode data;
try {
data = objectMapper.readTree(bytes);
} catch (Exception e) {
throw new SerializationException(e);
}
return data;
}
}

View File

@@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.json;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import com.fasterxml.jackson.databind.node.ObjectNode;
public class JsonSchema {
static final String ENVELOPE_SCHEMA_FIELD_NAME = "schema";
static final String ENVELOPE_PAYLOAD_FIELD_NAME = "payload";
static final String SCHEMA_TYPE_FIELD_NAME = "type";
static final String SCHEMA_OPTIONAL_FIELD_NAME = "optional";
static final String SCHEMA_NAME_FIELD_NAME = "name";
static final String SCHEMA_VERSION_FIELD_NAME = "version";
static final String SCHEMA_DOC_FIELD_NAME = "doc";
static final String SCHEMA_PARAMETERS_FIELD_NAME = "parameters";
static final String SCHEMA_DEFAULT_FIELD_NAME = "default";
static final String ARRAY_ITEMS_FIELD_NAME = "items";
static final String MAP_KEY_FIELD_NAME = "keys";
static final String MAP_VALUE_FIELD_NAME = "values";
static final String STRUCT_FIELDS_FIELD_NAME = "fields";
static final String STRUCT_FIELD_NAME_FIELD_NAME = "field";
static final String BOOLEAN_TYPE_NAME = "boolean";
static final ObjectNode BOOLEAN_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, BOOLEAN_TYPE_NAME);
static final String INT8_TYPE_NAME = "int8";
static final ObjectNode INT8_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT8_TYPE_NAME);
static final String INT16_TYPE_NAME = "int16";
static final ObjectNode INT16_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT16_TYPE_NAME);
static final String INT32_TYPE_NAME = "int32";
static final ObjectNode INT32_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT32_TYPE_NAME);
static final String INT64_TYPE_NAME = "int64";
static final ObjectNode INT64_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT64_TYPE_NAME);
static final String FLOAT_TYPE_NAME = "float";
static final ObjectNode FLOAT_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, FLOAT_TYPE_NAME);
static final String DOUBLE_TYPE_NAME = "double";
static final ObjectNode DOUBLE_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, DOUBLE_TYPE_NAME);
static final String BYTES_TYPE_NAME = "bytes";
static final ObjectNode BYTES_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, BYTES_TYPE_NAME);
static final String STRING_TYPE_NAME = "string";
static final ObjectNode STRING_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, STRING_TYPE_NAME);
static final String ARRAY_TYPE_NAME = "array";
static final String MAP_TYPE_NAME = "map";
static final String STRUCT_TYPE_NAME = "struct";
public static ObjectNode envelope(JsonNode schema, JsonNode payload) {
ObjectNode result = JsonNodeFactory.instance.objectNode();
result.set(ENVELOPE_SCHEMA_FIELD_NAME, schema);
result.set(ENVELOPE_PAYLOAD_FIELD_NAME, payload);
return result;
}
static class Envelope {
public JsonNode schema;
public JsonNode payload;
public Envelope(JsonNode schema, JsonNode payload) {
this.schema = schema;
this.payload = payload;
}
public ObjectNode toJsonNode() {
return envelope(schema, payload);
}
}
}

View File

@@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.json;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.serialization.Serializer;
import java.util.Collections;
import java.util.Set;
/**
* Serialize Jackson JsonNode tree model objects to UTF-8 JSON. Using the tree model allows handling arbitrarily
* structured data without corresponding Java classes. This serializer also supports Connect schemas.
*/
public class JsonSerializer implements Serializer<JsonNode> {
private final ObjectMapper objectMapper = new ObjectMapper();
/**
* Default constructor needed by Kafka
*/
public JsonSerializer() {
this(Collections.emptySet(), JsonNodeFactory.withExactBigDecimals(true));
}
/**
* A constructor that additionally specifies some {@link SerializationFeature}
* for the serializer
*
* @param serializationFeatures the specified serialization features
* @param jsonNodeFactory the json node factory to use.
*/
JsonSerializer(
final Set<SerializationFeature> serializationFeatures,
final JsonNodeFactory jsonNodeFactory
) {
serializationFeatures.forEach(objectMapper::enable);
objectMapper.setNodeFactory(jsonNodeFactory);
}
@Override
public byte[] serialize(String topic, JsonNode data) {
if (data == null)
return null;
try {
return objectMapper.writeValueAsBytes(data);
} catch (Exception e) {
throw new SerializationException("Error serializing JSON message", e);
}
}
}

View File

@@ -0,0 +1,39 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.json;
import static org.junit.Assert.assertEquals;
import java.util.HashMap;
import java.util.Map;
import org.apache.kafka.connect.storage.ConverterConfig;
import org.apache.kafka.connect.storage.ConverterType;
import org.junit.Test;
public class JsonConverterConfigTest {
@Test
public void shouldBeCaseInsensitiveForDecimalFormatConfig() {
final Map<String, Object> configValues = new HashMap<>();
configValues.put(ConverterConfig.TYPE_CONFIG, ConverterType.KEY.getName());
configValues.put(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, "NuMeRiC");
final JsonConverterConfig config = new JsonConverterConfig(configValues);
assertEquals(config.decimalFormat(), DecimalFormat.NUMERIC);
}
}

View File

@@ -0,0 +1,906 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.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;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.data.Date;
import org.apache.kafka.connect.data.Decimal;
import com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.kafka.common.cache.Cache;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.data.Time;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.errors.DataException;
import org.junit.Before;
import org.junit.Test;
import org.powermock.reflect.Whitebox;
import java.io.File;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.net.URISyntaxException;
import java.net.URL;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Calendar;
import java.util.Collections;
import java.util.GregorianCalendar;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.TimeZone;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class JsonConverterTest {
private static final String TOPIC = "topic";
private final ObjectMapper objectMapper = new ObjectMapper()
.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS)
.setNodeFactory(JsonNodeFactory.withExactBigDecimals(true));
private final JsonConverter converter = new JsonConverter();
@Before
public void setUp() {
converter.configure(Collections.emptyMap(), false);
}
// Schema metadata
@Test
public void testConnectSchemaMetadataTranslation() {
// this validates the non-type fields are translated and handled properly
assertEquals(new SchemaAndValue(Schema.BOOLEAN_SCHEMA, true), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes()));
assertEquals(new SchemaAndValue(Schema.OPTIONAL_BOOLEAN_SCHEMA, null), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": true }, \"payload\": null }".getBytes()));
assertEquals(new SchemaAndValue(SchemaBuilder.bool().defaultValue(true).build(), true),
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"default\": true }, \"payload\": null }".getBytes()));
assertEquals(new SchemaAndValue(SchemaBuilder.bool().required().name("bool").version(2).doc("the documentation").parameter("foo", "bar").build(), true),
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": false, \"name\": \"bool\", \"version\": 2, \"doc\": \"the documentation\", \"parameters\": { \"foo\": \"bar\" }}, \"payload\": true }".getBytes()));
}
// Schema types
@Test
public void booleanToConnect() {
assertEquals(new SchemaAndValue(Schema.BOOLEAN_SCHEMA, true), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes()));
assertEquals(new SchemaAndValue(Schema.BOOLEAN_SCHEMA, false), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": false }".getBytes()));
}
@Test
public void byteToConnect() {
assertEquals(new SchemaAndValue(Schema.INT8_SCHEMA, (byte) 12), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"int8\" }, \"payload\": 12 }".getBytes()));
}
@Test
public void shortToConnect() {
assertEquals(new SchemaAndValue(Schema.INT16_SCHEMA, (short) 12), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"int16\" }, \"payload\": 12 }".getBytes()));
}
@Test
public void intToConnect() {
assertEquals(new SchemaAndValue(Schema.INT32_SCHEMA, 12), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"int32\" }, \"payload\": 12 }".getBytes()));
}
@Test
public void longToConnect() {
assertEquals(new SchemaAndValue(Schema.INT64_SCHEMA, 12L), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"int64\" }, \"payload\": 12 }".getBytes()));
assertEquals(new SchemaAndValue(Schema.INT64_SCHEMA, 4398046511104L), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"int64\" }, \"payload\": 4398046511104 }".getBytes()));
}
@Test
public void floatToConnect() {
assertEquals(new SchemaAndValue(Schema.FLOAT32_SCHEMA, 12.34f), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"float\" }, \"payload\": 12.34 }".getBytes()));
}
@Test
public void doubleToConnect() {
assertEquals(new SchemaAndValue(Schema.FLOAT64_SCHEMA, 12.34), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"double\" }, \"payload\": 12.34 }".getBytes()));
}
@Test
public void bytesToConnect() throws UnsupportedEncodingException {
ByteBuffer reference = ByteBuffer.wrap("test-string".getBytes("UTF-8"));
String msg = "{ \"schema\": { \"type\": \"bytes\" }, \"payload\": \"dGVzdC1zdHJpbmc=\" }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
ByteBuffer converted = ByteBuffer.wrap((byte[]) schemaAndValue.value());
assertEquals(reference, converted);
}
@Test
public void stringToConnect() {
assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes()));
}
@Test
public void arrayToConnect() {
byte[] arrayJson = "{ \"schema\": { \"type\": \"array\", \"items\": { \"type\" : \"int32\" } }, \"payload\": [1, 2, 3] }".getBytes();
assertEquals(new SchemaAndValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList(1, 2, 3)), converter.toConnectData(TOPIC, arrayJson));
}
@Test
public void mapToConnectStringKeys() {
byte[] mapJson = "{ \"schema\": { \"type\": \"map\", \"keys\": { \"type\" : \"string\" }, \"values\": { \"type\" : \"int32\" } }, \"payload\": { \"key1\": 12, \"key2\": 15} }".getBytes();
Map<String, Integer> expected = new HashMap<>();
expected.put("key1", 12);
expected.put("key2", 15);
assertEquals(new SchemaAndValue(SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).build(), expected), converter.toConnectData(TOPIC, mapJson));
}
@Test
public void mapToConnectNonStringKeys() {
byte[] mapJson = "{ \"schema\": { \"type\": \"map\", \"keys\": { \"type\" : \"int32\" }, \"values\": { \"type\" : \"int32\" } }, \"payload\": [ [1, 12], [2, 15] ] }".getBytes();
Map<Integer, Integer> expected = new HashMap<>();
expected.put(1, 12);
expected.put(2, 15);
assertEquals(new SchemaAndValue(SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.INT32_SCHEMA).build(), expected), converter.toConnectData(TOPIC, mapJson));
}
@Test
public void structToConnect() {
byte[] structJson = "{ \"schema\": { \"type\": \"struct\", \"fields\": [{ \"field\": \"field1\", \"type\": \"boolean\" }, { \"field\": \"field2\", \"type\": \"string\" }] }, \"payload\": { \"field1\": true, \"field2\": \"string\" } }".getBytes();
Schema expectedSchema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA).field("field2", Schema.STRING_SCHEMA).build();
Struct expected = new Struct(expectedSchema).put("field1", true).put("field2", "string");
SchemaAndValue converted = converter.toConnectData(TOPIC, structJson);
assertEquals(new SchemaAndValue(expectedSchema, expected), converted);
}
@Test
public void structWithOptionalFieldToConnect() {
byte[] structJson = "{ \"schema\": { \"type\": \"struct\", \"fields\": [{ \"field\":\"optional\", \"type\": \"string\", \"optional\": true }, { \"field\": \"required\", \"type\": \"string\" }] }, \"payload\": { \"required\": \"required\" } }".getBytes();
Schema expectedSchema = SchemaBuilder.struct().field("optional", Schema.OPTIONAL_STRING_SCHEMA).field("required", Schema.STRING_SCHEMA).build();
Struct expected = new Struct(expectedSchema).put("required", "required");
SchemaAndValue converted = converter.toConnectData(TOPIC, structJson);
assertEquals(new SchemaAndValue(expectedSchema, expected), converted);
}
@Test
public void nullToConnect() {
// When schemas are enabled, trying to decode a tombstone should be an empty envelope
// the behavior is the same as when the json is "{ "schema": null, "payload": null }"
// to keep compatibility with the record
SchemaAndValue converted = converter.toConnectData(TOPIC, null);
assertEquals(SchemaAndValue.NULL, converted);
}
@Test
public void nullSchemaPrimitiveToConnect() {
SchemaAndValue converted = converter.toConnectData(TOPIC, "{ \"schema\": null, \"payload\": null }".getBytes());
assertEquals(SchemaAndValue.NULL, converted);
converted = converter.toConnectData(TOPIC, "{ \"schema\": null, \"payload\": true }".getBytes());
assertEquals(new SchemaAndValue(null, true), converted);
// Integers: Connect has more data types, and JSON unfortunately mixes all number types. We try to preserve
// info as best we can, so we always use the largest integer and floating point numbers we can and have Jackson
// determine if it's an integer or not
converted = converter.toConnectData(TOPIC, "{ \"schema\": null, \"payload\": 12 }".getBytes());
assertEquals(new SchemaAndValue(null, 12L), converted);
converted = converter.toConnectData(TOPIC, "{ \"schema\": null, \"payload\": 12.24 }".getBytes());
assertEquals(new SchemaAndValue(null, 12.24), converted);
converted = converter.toConnectData(TOPIC, "{ \"schema\": null, \"payload\": \"a string\" }".getBytes());
assertEquals(new SchemaAndValue(null, "a string"), converted);
converted = converter.toConnectData(TOPIC, "{ \"schema\": null, \"payload\": [1, \"2\", 3] }".getBytes());
assertEquals(new SchemaAndValue(null, Arrays.asList(1L, "2", 3L)), converted);
converted = converter.toConnectData(TOPIC, "{ \"schema\": null, \"payload\": { \"field1\": 1, \"field2\": 2} }".getBytes());
Map<String, Long> obj = new HashMap<>();
obj.put("field1", 1L);
obj.put("field2", 2L);
assertEquals(new SchemaAndValue(null, obj), converted);
}
@Test
public void decimalToConnect() {
Schema schema = Decimal.schema(2);
BigDecimal reference = new BigDecimal(new BigInteger("156"), 2);
// Payload is base64 encoded byte[]{0, -100}, which is the two's complement encoding of 156.
String msg = "{ \"schema\": { \"type\": \"bytes\", \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"2\" } }, \"payload\": \"AJw=\" }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
BigDecimal converted = (BigDecimal) schemaAndValue.value();
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, converted);
}
@Test
public void decimalToConnectOptional() {
Schema schema = Decimal.builder(2).optional().schema();
String msg = "{ \"schema\": { \"type\": \"bytes\", \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"optional\": true, \"parameters\": { \"scale\": \"2\" } }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertNull(schemaAndValue.value());
}
@Test
public void decimalToConnectWithDefaultValue() {
BigDecimal reference = new BigDecimal(new BigInteger("156"), 2);
Schema schema = Decimal.builder(2).defaultValue(reference).build();
String msg = "{ \"schema\": { \"type\": \"bytes\", \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"default\": \"AJw=\", \"parameters\": { \"scale\": \"2\" } }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void decimalToConnectOptionalWithDefaultValue() {
BigDecimal reference = new BigDecimal(new BigInteger("156"), 2);
Schema schema = Decimal.builder(2).optional().defaultValue(reference).build();
String msg = "{ \"schema\": { \"type\": \"bytes\", \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"optional\": true, \"default\": \"AJw=\", \"parameters\": { \"scale\": \"2\" } }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void numericDecimalToConnect() {
BigDecimal reference = new BigDecimal(new BigInteger("156"), 2);
Schema schema = Decimal.schema(2);
String msg = "{ \"schema\": { \"type\": \"bytes\", \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"2\" } }, \"payload\": 1.56 }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void numericDecimalWithTrailingZerosToConnect() {
BigDecimal reference = new BigDecimal(new BigInteger("15600"), 4);
Schema schema = Decimal.schema(4);
String msg = "{ \"schema\": { \"type\": \"bytes\", \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"4\" } }, \"payload\": 1.5600 }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void highPrecisionNumericDecimalToConnect() {
// this number is too big to be kept in a float64!
BigDecimal reference = new BigDecimal("1.23456789123456789");
Schema schema = Decimal.schema(17);
String msg = "{ \"schema\": { \"type\": \"bytes\", \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"17\" } }, \"payload\": 1.23456789123456789 }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void dateToConnect() {
Schema schema = Date.SCHEMA;
GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0);
calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
calendar.add(Calendar.DATE, 10000);
java.util.Date reference = calendar.getTime();
String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.connect.data.Date\", \"version\": 1 }, \"payload\": 10000 }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
java.util.Date converted = (java.util.Date) schemaAndValue.value();
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, converted);
}
@Test
public void dateToConnectOptional() {
Schema schema = Date.builder().optional().schema();
String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.connect.data.Date\", \"version\": 1, \"optional\": true }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertNull(schemaAndValue.value());
}
@Test
public void dateToConnectWithDefaultValue() {
java.util.Date reference = new java.util.Date(0);
Schema schema = Date.builder().defaultValue(reference).schema();
String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.connect.data.Date\", \"version\": 1, \"default\": 0 }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void dateToConnectOptionalWithDefaultValue() {
java.util.Date reference = new java.util.Date(0);
Schema schema = Date.builder().optional().defaultValue(reference).schema();
String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.connect.data.Date\", \"version\": 1, \"optional\": true, \"default\": 0 }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void timeToConnect() {
Schema schema = Time.SCHEMA;
GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0);
calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
calendar.add(Calendar.MILLISECOND, 14400000);
java.util.Date reference = calendar.getTime();
String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.connect.data.Time\", \"version\": 1 }, \"payload\": 14400000 }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
java.util.Date converted = (java.util.Date) schemaAndValue.value();
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, converted);
}
@Test
public void timeToConnectOptional() {
Schema schema = Time.builder().optional().schema();
String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.connect.data.Time\", \"version\": 1, \"optional\": true }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertNull(schemaAndValue.value());
}
@Test
public void timeToConnectWithDefaultValue() {
java.util.Date reference = new java.util.Date(0);
Schema schema = Time.builder().defaultValue(reference).schema();
String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.connect.data.Time\", \"version\": 1, \"default\": 0 }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void timeToConnectOptionalWithDefaultValue() {
java.util.Date reference = new java.util.Date(0);
Schema schema = Time.builder().optional().defaultValue(reference).schema();
String msg = "{ \"schema\": { \"type\": \"int32\", \"name\": \"org.apache.kafka.connect.data.Time\", \"version\": 1, \"optional\": true, \"default\": 0 }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, schemaAndValue.value());
}
@Test
public void timestampToConnect() {
Schema schema = Timestamp.SCHEMA;
GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0);
calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
calendar.add(Calendar.MILLISECOND, 2000000000);
calendar.add(Calendar.MILLISECOND, 2000000000);
java.util.Date reference = calendar.getTime();
String msg = "{ \"schema\": { \"type\": \"int64\", \"name\": \"org.apache.kafka.connect.data.Timestamp\", \"version\": 1 }, \"payload\": 4000000000 }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
java.util.Date converted = (java.util.Date) schemaAndValue.value();
assertEquals(schema, schemaAndValue.schema());
assertEquals(reference, converted);
}
@Test
public void timestampToConnectOptional() {
Schema schema = Timestamp.builder().optional().schema();
String msg = "{ \"schema\": { \"type\": \"int64\", \"name\": \"org.apache.kafka.connect.data.Timestamp\", \"version\": 1, \"optional\": true }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertNull(schemaAndValue.value());
}
@Test
public void timestampToConnectWithDefaultValue() {
Schema schema = Timestamp.builder().defaultValue(new java.util.Date(42)).schema();
String msg = "{ \"schema\": { \"type\": \"int64\", \"name\": \"org.apache.kafka.connect.data.Timestamp\", \"version\": 1, \"default\": 42 }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(new java.util.Date(42), schemaAndValue.value());
}
@Test
public void timestampToConnectOptionalWithDefaultValue() {
Schema schema = Timestamp.builder().optional().defaultValue(new java.util.Date(42)).schema();
String msg = "{ \"schema\": { \"type\": \"int64\", \"name\": \"org.apache.kafka.connect.data.Timestamp\", \"version\": 1, \"optional\": true, \"default\": 42 }, \"payload\": null }";
SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes());
assertEquals(schema, schemaAndValue.schema());
assertEquals(new java.util.Date(42), schemaAndValue.value());
}
// Schema metadata
@Test
public void testJsonSchemaMetadataTranslation() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.BOOLEAN_SCHEMA, true));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"boolean\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue());
converted = parse(converter.fromConnectData(TOPIC, Schema.OPTIONAL_BOOLEAN_SCHEMA, null));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"boolean\", \"optional\": true }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertTrue(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isNull());
converted = parse(converter.fromConnectData(TOPIC, SchemaBuilder.bool().defaultValue(true).build(), true));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"boolean\", \"optional\": false, \"default\": true }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue());
converted = parse(converter.fromConnectData(TOPIC, SchemaBuilder.bool().required().name("bool").version(3).doc("the documentation").parameter("foo", "bar").build(), true));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"boolean\", \"optional\": false, \"name\": \"bool\", \"version\": 3, \"doc\": \"the documentation\", \"parameters\": { \"foo\": \"bar\" }}"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue());
}
@Test
public void testCacheSchemaToConnectConversion() {
Cache<JsonNode, Schema> cache = Whitebox.getInternalState(converter, "toConnectSchemaCache");
assertEquals(0, cache.size());
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes());
assertEquals(1, cache.size());
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes());
assertEquals(1, cache.size());
// Different schema should also get cached
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": true }, \"payload\": true }".getBytes());
assertEquals(2, cache.size());
// Even equivalent, but different JSON encoding of schema, should get different cache entry
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": false }, \"payload\": true }".getBytes());
assertEquals(3, cache.size());
}
// Schema types
@Test
public void booleanToJson() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.BOOLEAN_SCHEMA, true));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"boolean\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue());
}
@Test
public void byteToJson() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.INT8_SCHEMA, (byte) 12));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"int8\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(12, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).intValue());
}
@Test
public void shortToJson() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.INT16_SCHEMA, (short) 12));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"int16\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(12, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).intValue());
}
@Test
public void intToJson() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.INT32_SCHEMA, 12));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"int32\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(12, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).intValue());
}
@Test
public void longToJson() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.INT64_SCHEMA, 4398046511104L));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"int64\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(4398046511104L, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).longValue());
}
@Test
public void floatToJson() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.FLOAT32_SCHEMA, 12.34f));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"float\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(12.34f, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).floatValue(), 0.001);
}
@Test
public void doubleToJson() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.FLOAT64_SCHEMA, 12.34));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"double\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(12.34, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).doubleValue(), 0.001);
}
@Test
public void bytesToJson() throws IOException {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, "test-string".getBytes()));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"bytes\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(ByteBuffer.wrap("test-string".getBytes()),
ByteBuffer.wrap(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).binaryValue()));
}
@Test
public void stringToJson() {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Schema.STRING_SCHEMA, "test-string"));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"string\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals("test-string", converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).textValue());
}
@Test
public void arrayToJson() {
Schema int32Array = SchemaBuilder.array(Schema.INT32_SCHEMA).build();
JsonNode converted = parse(converter.fromConnectData(TOPIC, int32Array, Arrays.asList(1, 2, 3)));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"array\", \"items\": { \"type\": \"int32\", \"optional\": false }, \"optional\": false }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(JsonNodeFactory.instance.arrayNode().add(1).add(2).add(3),
converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
@Test
public void mapToJsonStringKeys() {
Schema stringIntMap = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).build();
Map<String, Integer> input = new HashMap<>();
input.put("key1", 12);
input.put("key2", 15);
JsonNode converted = parse(converter.fromConnectData(TOPIC, stringIntMap, input));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"map\", \"keys\": { \"type\" : \"string\", \"optional\": false }, \"values\": { \"type\" : \"int32\", \"optional\": false }, \"optional\": false }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(JsonNodeFactory.instance.objectNode().put("key1", 12).put("key2", 15),
converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
@Test
public void mapToJsonNonStringKeys() {
Schema intIntMap = SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.INT32_SCHEMA).build();
Map<Integer, Integer> input = new HashMap<>();
input.put(1, 12);
input.put(2, 15);
JsonNode converted = parse(converter.fromConnectData(TOPIC, intIntMap, input));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"map\", \"keys\": { \"type\" : \"int32\", \"optional\": false }, \"values\": { \"type\" : \"int32\", \"optional\": false }, \"optional\": false }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertTrue(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isArray());
ArrayNode payload = (ArrayNode) converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME);
assertEquals(2, payload.size());
Set<JsonNode> payloadEntries = new HashSet<>();
for (JsonNode elem : payload)
payloadEntries.add(elem);
assertEquals(new HashSet<>(Arrays.asList(JsonNodeFactory.instance.arrayNode().add(1).add(12),
JsonNodeFactory.instance.arrayNode().add(2).add(15))),
payloadEntries
);
}
@Test
public void structToJson() {
Schema schema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA).field("field2", Schema.STRING_SCHEMA).field("field3", Schema.STRING_SCHEMA).field("field4", Schema.BOOLEAN_SCHEMA).build();
Struct input = new Struct(schema).put("field1", true).put("field2", "string2").put("field3", "string3").put("field4", false);
JsonNode converted = parse(converter.fromConnectData(TOPIC, schema, input));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"struct\", \"optional\": false, \"fields\": [{ \"field\": \"field1\", \"type\": \"boolean\", \"optional\": false }, { \"field\": \"field2\", \"type\": \"string\", \"optional\": false }, { \"field\": \"field3\", \"type\": \"string\", \"optional\": false }, { \"field\": \"field4\", \"type\": \"boolean\", \"optional\": false }] }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(JsonNodeFactory.instance.objectNode()
.put("field1", true)
.put("field2", "string2")
.put("field3", "string3")
.put("field4", false),
converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
@Test
public void structSchemaIdentical() {
Schema schema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA)
.field("field2", Schema.STRING_SCHEMA)
.field("field3", Schema.STRING_SCHEMA)
.field("field4", Schema.BOOLEAN_SCHEMA).build();
Schema inputSchema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA)
.field("field2", Schema.STRING_SCHEMA)
.field("field3", Schema.STRING_SCHEMA)
.field("field4", Schema.BOOLEAN_SCHEMA).build();
Struct input = new Struct(inputSchema).put("field1", true).put("field2", "string2").put("field3", "string3").put("field4", false);
assertStructSchemaEqual(schema, input);
}
@Test
public void decimalToJson() throws IOException {
JsonNode converted = parse(converter.fromConnectData(TOPIC, Decimal.schema(2), new BigDecimal(new BigInteger("156"), 2)));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"bytes\", \"optional\": false, \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"2\" } }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertTrue("expected node to be base64 text", converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isTextual());
assertArrayEquals(new byte[]{0, -100}, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).binaryValue());
}
@Test
public void decimalToNumericJson() {
converter.configure(Collections.singletonMap(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, DecimalFormat.NUMERIC.name()), false);
JsonNode converted = parse(converter.fromConnectData(TOPIC, Decimal.schema(2), new BigDecimal(new BigInteger("156"), 2)));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"bytes\", \"optional\": false, \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"2\" } }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertTrue("expected node to be numeric", converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isNumber());
assertEquals(new BigDecimal("1.56"), converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).decimalValue());
}
@Test
public void decimalWithTrailingZerosToNumericJson() {
converter.configure(Collections.singletonMap(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, DecimalFormat.NUMERIC.name()), false);
JsonNode converted = parse(converter.fromConnectData(TOPIC, Decimal.schema(4), new BigDecimal(new BigInteger("15600"), 4)));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"bytes\", \"optional\": false, \"name\": \"org.apache.kafka.connect.data.Decimal\", \"version\": 1, \"parameters\": { \"scale\": \"4\" } }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertTrue("expected node to be numeric", converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isNumber());
assertEquals(new BigDecimal("1.5600"), converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).decimalValue());
}
@Test
public void decimalToJsonWithoutSchema() {
assertThrows(
"expected data exception when serializing BigDecimal without schema",
DataException.class,
() -> converter.fromConnectData(TOPIC, null, new BigDecimal(new BigInteger("156"), 2)));
}
@Test
public void dateToJson() {
GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0);
calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
calendar.add(Calendar.DATE, 10000);
java.util.Date date = calendar.getTime();
JsonNode converted = parse(converter.fromConnectData(TOPIC, Date.SCHEMA, date));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"int32\", \"optional\": false, \"name\": \"org.apache.kafka.connect.data.Date\", \"version\": 1 }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
JsonNode payload = converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME);
assertTrue(payload.isInt());
assertEquals(10000, payload.intValue());
}
@Test
public void timeToJson() {
GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0);
calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
calendar.add(Calendar.MILLISECOND, 14400000);
java.util.Date date = calendar.getTime();
JsonNode converted = parse(converter.fromConnectData(TOPIC, Time.SCHEMA, date));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"int32\", \"optional\": false, \"name\": \"org.apache.kafka.connect.data.Time\", \"version\": 1 }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
JsonNode payload = converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME);
assertTrue(payload.isInt());
assertEquals(14400000, payload.longValue());
}
@Test
public void timestampToJson() {
GregorianCalendar calendar = new GregorianCalendar(1970, Calendar.JANUARY, 1, 0, 0, 0);
calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
calendar.add(Calendar.MILLISECOND, 2000000000);
calendar.add(Calendar.MILLISECOND, 2000000000);
java.util.Date date = calendar.getTime();
JsonNode converted = parse(converter.fromConnectData(TOPIC, Timestamp.SCHEMA, date));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"int64\", \"optional\": false, \"name\": \"org.apache.kafka.connect.data.Timestamp\", \"version\": 1 }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
JsonNode payload = converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME);
assertTrue(payload.isLong());
assertEquals(4000000000L, payload.longValue());
}
@Test
public void nullSchemaAndPrimitiveToJson() {
// This still needs to do conversion of data, null schema means "anything goes"
JsonNode converted = parse(converter.fromConnectData(TOPIC, null, true));
validateEnvelopeNullSchema(converted);
assertTrue(converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull());
assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue());
}
@Test
public void nullSchemaAndArrayToJson() {
// This still needs to do conversion of data, null schema means "anything goes". Make sure we mix and match
// types to verify conversion still works.
JsonNode converted = parse(converter.fromConnectData(TOPIC, null, Arrays.asList(1, "string", true)));
validateEnvelopeNullSchema(converted);
assertTrue(converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull());
assertEquals(JsonNodeFactory.instance.arrayNode().add(1).add("string").add(true),
converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
@Test
public void nullSchemaAndMapToJson() {
// This still needs to do conversion of data, null schema means "anything goes". Make sure we mix and match
// types to verify conversion still works.
Map<String, Object> input = new HashMap<>();
input.put("key1", 12);
input.put("key2", "string");
input.put("key3", true);
JsonNode converted = parse(converter.fromConnectData(TOPIC, null, input));
validateEnvelopeNullSchema(converted);
assertTrue(converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull());
assertEquals(JsonNodeFactory.instance.objectNode().put("key1", 12).put("key2", "string").put("key3", true),
converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
@Test
public void nullSchemaAndMapNonStringKeysToJson() {
// This still needs to do conversion of data, null schema means "anything goes". Make sure we mix and match
// types to verify conversion still works.
Map<Object, Object> input = new HashMap<>();
input.put("string", 12);
input.put(52, "string");
input.put(false, true);
JsonNode converted = parse(converter.fromConnectData(TOPIC, null, input));
validateEnvelopeNullSchema(converted);
assertTrue(converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull());
assertTrue(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).isArray());
ArrayNode payload = (ArrayNode) converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME);
assertEquals(3, payload.size());
Set<JsonNode> payloadEntries = new HashSet<>();
for (JsonNode elem : payload)
payloadEntries.add(elem);
assertEquals(new HashSet<>(Arrays.asList(JsonNodeFactory.instance.arrayNode().add("string").add(12),
JsonNodeFactory.instance.arrayNode().add(52).add("string"),
JsonNodeFactory.instance.arrayNode().add(false).add(true))),
payloadEntries
);
}
@Test
public void nullSchemaAndNullValueToJson() {
// This characterizes the production of tombstone messages when Json schemas is enabled
Map<String, Boolean> props = Collections.singletonMap("schemas.enable", true);
converter.configure(props, true);
byte[] converted = converter.fromConnectData(TOPIC, null, null);
assertNull(converted);
}
@Test
public void nullValueToJson() {
// This characterizes the production of tombstone messages when Json schemas is not enabled
Map<String, Boolean> props = Collections.singletonMap("schemas.enable", false);
converter.configure(props, true);
byte[] converted = converter.fromConnectData(TOPIC, null, null);
assertNull(converted);
}
@Test(expected = DataException.class)
public void mismatchSchemaJson() {
// If we have mismatching schema info, we should properly convert to a DataException
converter.fromConnectData(TOPIC, Schema.FLOAT64_SCHEMA, true);
}
@Test
public void noSchemaToConnect() {
Map<String, Boolean> props = Collections.singletonMap("schemas.enable", false);
converter.configure(props, true);
assertEquals(new SchemaAndValue(null, true), converter.toConnectData(TOPIC, "true".getBytes()));
}
@Test
public void noSchemaToJson() {
Map<String, Boolean> props = Collections.singletonMap("schemas.enable", false);
converter.configure(props, true);
JsonNode converted = parse(converter.fromConnectData(TOPIC, null, true));
assertTrue(converted.isBoolean());
assertEquals(true, converted.booleanValue());
}
@Test
public void testCacheSchemaToJsonConversion() {
Cache<Schema, ObjectNode> cache = Whitebox.getInternalState(converter, "fromConnectSchemaCache");
assertEquals(0, cache.size());
// Repeated conversion of the same schema, even if the schema object is different should return the same Java
// object
converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true);
assertEquals(1, cache.size());
converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true);
assertEquals(1, cache.size());
// Validate that a similar, but different schema correctly returns a different schema.
converter.fromConnectData(TOPIC, SchemaBuilder.bool().optional().build(), true);
assertEquals(2, cache.size());
}
@Test
public void testJsonSchemaCacheSizeFromConfigFile() throws URISyntaxException, IOException {
URL url = getClass().getResource("/connect-test.properties");
File propFile = new File(url.toURI());
String workerPropsFile = propFile.getAbsolutePath();
Map<String, String> workerProps = !workerPropsFile.isEmpty() ?
Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.<String, String>emptyMap();
JsonConverter rc = new JsonConverter();
rc.configure(workerProps, false);
}
// Note: the header conversion methods delegates to the data conversion methods, which are tested above.
// The following simply verify that the delegation works.
@Test
public void testStringHeaderToJson() {
JsonNode converted = parse(converter.fromConnectHeader(TOPIC, "headerName", Schema.STRING_SCHEMA, "test-string"));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"string\", \"optional\": false }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals("test-string", converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).textValue());
}
@Test
public void stringHeaderToConnect() {
assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectHeader(TOPIC, "headerName", "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes()));
}
private JsonNode parse(byte[] json) {
try {
return objectMapper.readTree(json);
} catch (IOException e) {
fail("IOException during JSON parse: " + e.getMessage());
throw new RuntimeException("failed");
}
}
private JsonNode parse(String json) {
try {
return objectMapper.readTree(json);
} catch (IOException e) {
fail("IOException during JSON parse: " + e.getMessage());
throw new RuntimeException("failed");
}
}
private void validateEnvelope(JsonNode env) {
assertNotNull(env);
assertTrue(env.isObject());
assertEquals(2, env.size());
assertTrue(env.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertTrue(env.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isObject());
assertTrue(env.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
private void validateEnvelopeNullSchema(JsonNode env) {
assertNotNull(env);
assertTrue(env.isObject());
assertEquals(2, env.size());
assertTrue(env.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertTrue(env.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull());
assertTrue(env.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
private void assertStructSchemaEqual(Schema schema, Struct struct) {
converter.fromConnectData(TOPIC, schema, struct);
assertEquals(schema, struct.schema());
}
}

View File

@@ -0,0 +1,17 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (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.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
schemas.cache.size=1