mirror of https://github.com/apache/kafka.git
KAFKA-4714; Flatten and Cast single message transforms (KIP-66)
Author: Ewen Cheslack-Postava <me@ewencp.org> Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Shikhar Bhushan <shikhar@confluent.io>, Jason Gustafson <jason@confluent.io> Closes #2458 from ewencp/kafka-3209-even-more-transforms
This commit is contained in:
parent
ebc7f7caae
commit
1cea4d8f5a
|
|
@ -75,7 +75,7 @@ public class SchemaBuilder implements Schema {
|
||||||
// Additional parameters for logical types.
|
// Additional parameters for logical types.
|
||||||
private Map<String, String> parameters;
|
private Map<String, String> parameters;
|
||||||
|
|
||||||
private SchemaBuilder(Type type) {
|
public SchemaBuilder(Type type) {
|
||||||
this.type = type;
|
this.type = type;
|
||||||
if (type == Type.STRUCT) {
|
if (type == Type.STRUCT) {
|
||||||
fields = new LinkedHashMap<>();
|
fields = new LinkedHashMap<>();
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,9 @@
|
||||||
package org.apache.kafka.connect.tools;
|
package org.apache.kafka.connect.tools;
|
||||||
|
|
||||||
import org.apache.kafka.common.config.ConfigDef;
|
import org.apache.kafka.common.config.ConfigDef;
|
||||||
|
import org.apache.kafka.connect.transforms.Cast;
|
||||||
import org.apache.kafka.connect.transforms.ExtractField;
|
import org.apache.kafka.connect.transforms.ExtractField;
|
||||||
|
import org.apache.kafka.connect.transforms.Flatten;
|
||||||
import org.apache.kafka.connect.transforms.HoistField;
|
import org.apache.kafka.connect.transforms.HoistField;
|
||||||
import org.apache.kafka.connect.transforms.InsertField;
|
import org.apache.kafka.connect.transforms.InsertField;
|
||||||
import org.apache.kafka.connect.transforms.MaskField;
|
import org.apache.kafka.connect.transforms.MaskField;
|
||||||
|
|
@ -54,7 +56,9 @@ public class TransformationDoc {
|
||||||
new DocInfo(ExtractField.class.getName(), ExtractField.OVERVIEW_DOC, ExtractField.CONFIG_DEF),
|
new DocInfo(ExtractField.class.getName(), ExtractField.OVERVIEW_DOC, ExtractField.CONFIG_DEF),
|
||||||
new DocInfo(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.CONFIG_DEF),
|
new DocInfo(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.CONFIG_DEF),
|
||||||
new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF),
|
new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF),
|
||||||
new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF)
|
new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF),
|
||||||
|
new DocInfo(Flatten.class.getName(), Flatten.OVERVIEW_DOC, Flatten.CONFIG_DEF),
|
||||||
|
new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF)
|
||||||
);
|
);
|
||||||
|
|
||||||
private static void printTransformationHtml(PrintStream out, DocInfo docInfo) {
|
private static void printTransformationHtml(PrintStream out, DocInfo docInfo) {
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,417 @@
|
||||||
|
/*
|
||||||
|
* 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.transforms;
|
||||||
|
|
||||||
|
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.config.ConfigException;
|
||||||
|
import org.apache.kafka.connect.connector.ConnectRecord;
|
||||||
|
import org.apache.kafka.connect.data.ConnectSchema;
|
||||||
|
import org.apache.kafka.connect.data.Field;
|
||||||
|
import org.apache.kafka.connect.data.Schema;
|
||||||
|
import org.apache.kafka.connect.data.SchemaBuilder;
|
||||||
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
import org.apache.kafka.connect.errors.DataException;
|
||||||
|
import org.apache.kafka.connect.transforms.util.SchemaUtil;
|
||||||
|
import org.apache.kafka.connect.transforms.util.SimpleConfig;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Locale;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import static org.apache.kafka.connect.transforms.util.Requirements.requireMap;
|
||||||
|
import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct;
|
||||||
|
|
||||||
|
public abstract class Cast<R extends ConnectRecord<R>> implements Transformation<R> {
|
||||||
|
|
||||||
|
// TODO: Currently we only support top-level field casting. Ideally we could use a dotted notation in the spec to
|
||||||
|
// allow casting nested fields.
|
||||||
|
public static final String OVERVIEW_DOC =
|
||||||
|
"Cast fields or the entire key or value to a specific type, e.g. to force an integer field to a smaller "
|
||||||
|
+ "width. Only simple primitive types are supported -- integers, floats, boolean, and string. "
|
||||||
|
+ "<p/>Use the concrete transformation type designed for the record key (<code>" + Key.class.getName() + "</code>) "
|
||||||
|
+ "or value (<code>" + Value.class.getName() + "</code>).";
|
||||||
|
|
||||||
|
public static final String SPEC_CONFIG = "spec";
|
||||||
|
|
||||||
|
public static final ConfigDef CONFIG_DEF = new ConfigDef()
|
||||||
|
.define(SPEC_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new ConfigDef.Validator() {
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Override
|
||||||
|
public void ensureValid(String name, Object valueObject) {
|
||||||
|
List<String> value = (List<String>) valueObject;
|
||||||
|
if (value == null || value.isEmpty()) {
|
||||||
|
throw new ConfigException("Must specify at least one field to cast.");
|
||||||
|
}
|
||||||
|
parseFieldTypes(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "list of colon-delimited pairs, e.g. <code>foo:bar,abc:xyz</code>";
|
||||||
|
}
|
||||||
|
},
|
||||||
|
ConfigDef.Importance.HIGH,
|
||||||
|
"List of fields and the type to cast them to of the form field1:type,field2:type to cast fields of "
|
||||||
|
+ "Maps or Structs. A single type to cast the entire value. Valid types are int8, int16, int32, "
|
||||||
|
+ "int64, float32, float64, boolean, and string.");
|
||||||
|
|
||||||
|
private static final String PURPOSE = "cast types";
|
||||||
|
|
||||||
|
private static final Set<Schema.Type> SUPPORTED_CAST_TYPES = new HashSet<>(
|
||||||
|
Arrays.asList(Schema.Type.INT8, Schema.Type.INT16, Schema.Type.INT32, Schema.Type.INT64,
|
||||||
|
Schema.Type.FLOAT32, Schema.Type.FLOAT64, Schema.Type.BOOLEAN, Schema.Type.STRING)
|
||||||
|
);
|
||||||
|
|
||||||
|
// As a special case for casting the entire value (e.g. the incoming key is a int64 but you know it could be an
|
||||||
|
// int32 and want the smaller width), we use an otherwise invalid field name in the cast spec to track this.
|
||||||
|
private static final String WHOLE_VALUE_CAST = null;
|
||||||
|
|
||||||
|
private Map<String, Schema.Type> casts;
|
||||||
|
private Schema.Type wholeValueCastType;
|
||||||
|
private Cache<Schema, Schema> schemaUpdateCache;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void configure(Map<String, ?> props) {
|
||||||
|
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
|
||||||
|
casts = parseFieldTypes(config.getList(SPEC_CONFIG));
|
||||||
|
wholeValueCastType = casts.get(WHOLE_VALUE_CAST);
|
||||||
|
schemaUpdateCache = new SynchronizedCache<>(new LRUCache<Schema, Schema>(16));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public R apply(R record) {
|
||||||
|
if (operatingSchema(record) == null) {
|
||||||
|
return applySchemaless(record);
|
||||||
|
} else {
|
||||||
|
return applyWithSchema(record);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ConfigDef config() {
|
||||||
|
return CONFIG_DEF;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private R applySchemaless(R record) {
|
||||||
|
if (wholeValueCastType != null) {
|
||||||
|
return newRecord(record, null, castValueToType(operatingValue(record), wholeValueCastType));
|
||||||
|
}
|
||||||
|
|
||||||
|
final Map<String, Object> value = requireMap(operatingValue(record), PURPOSE);
|
||||||
|
final HashMap<String, Object> updatedValue = new HashMap<>(value);
|
||||||
|
for (Map.Entry<String, Schema.Type> fieldSpec : casts.entrySet()) {
|
||||||
|
String field = fieldSpec.getKey();
|
||||||
|
updatedValue.put(field, castValueToType(value.get(field), fieldSpec.getValue()));
|
||||||
|
}
|
||||||
|
return newRecord(record, null, updatedValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
private R applyWithSchema(R record) {
|
||||||
|
Schema valueSchema = operatingSchema(record);
|
||||||
|
Schema updatedSchema = getOrBuildSchema(valueSchema);
|
||||||
|
|
||||||
|
// Whole-record casting
|
||||||
|
if (wholeValueCastType != null)
|
||||||
|
return newRecord(record, updatedSchema, castValueToType(operatingValue(record), wholeValueCastType));
|
||||||
|
|
||||||
|
// Casting within a struct
|
||||||
|
final Struct value = requireStruct(operatingValue(record), PURPOSE);
|
||||||
|
|
||||||
|
final Struct updatedValue = new Struct(updatedSchema);
|
||||||
|
for (Field field : value.schema().fields()) {
|
||||||
|
final Object origFieldValue = value.get(field);
|
||||||
|
final Schema.Type targetType = casts.get(field.name());
|
||||||
|
final Object newFieldValue = targetType != null ? castValueToType(origFieldValue, targetType) : origFieldValue;
|
||||||
|
updatedValue.put(updatedSchema.field(field.name()), newFieldValue);
|
||||||
|
}
|
||||||
|
return newRecord(record, updatedSchema, updatedValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Schema getOrBuildSchema(Schema valueSchema) {
|
||||||
|
Schema updatedSchema = schemaUpdateCache.get(valueSchema);
|
||||||
|
if (updatedSchema != null)
|
||||||
|
return updatedSchema;
|
||||||
|
|
||||||
|
final SchemaBuilder builder;
|
||||||
|
if (wholeValueCastType != null) {
|
||||||
|
builder = SchemaUtil.copySchemaBasics(valueSchema, convertFieldType(wholeValueCastType));
|
||||||
|
} else {
|
||||||
|
builder = SchemaUtil.copySchemaBasics(valueSchema, SchemaBuilder.struct());
|
||||||
|
for (Field field : valueSchema.fields()) {
|
||||||
|
SchemaBuilder fieldBuilder =
|
||||||
|
convertFieldType(casts.containsKey(field.name()) ? casts.get(field.name()) : field.schema().type());
|
||||||
|
if (field.schema().isOptional())
|
||||||
|
fieldBuilder.optional();
|
||||||
|
if (field.schema().defaultValue() != null)
|
||||||
|
fieldBuilder.defaultValue(castValueToType(field.schema().defaultValue(), fieldBuilder.type()));
|
||||||
|
builder.field(field.name(), fieldBuilder.build());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (valueSchema.isOptional())
|
||||||
|
builder.optional();
|
||||||
|
if (valueSchema.defaultValue() != null)
|
||||||
|
builder.defaultValue(castValueToType(valueSchema.defaultValue(), builder.type()));
|
||||||
|
|
||||||
|
updatedSchema = builder.build();
|
||||||
|
schemaUpdateCache.put(valueSchema, updatedSchema);
|
||||||
|
return updatedSchema;
|
||||||
|
}
|
||||||
|
|
||||||
|
private SchemaBuilder convertFieldType(Schema.Type type) {
|
||||||
|
switch (type) {
|
||||||
|
case INT8:
|
||||||
|
return SchemaBuilder.int8();
|
||||||
|
case INT16:
|
||||||
|
return SchemaBuilder.int16();
|
||||||
|
case INT32:
|
||||||
|
return SchemaBuilder.int32();
|
||||||
|
case INT64:
|
||||||
|
return SchemaBuilder.int64();
|
||||||
|
case FLOAT32:
|
||||||
|
return SchemaBuilder.float32();
|
||||||
|
case FLOAT64:
|
||||||
|
return SchemaBuilder.float64();
|
||||||
|
case BOOLEAN:
|
||||||
|
return SchemaBuilder.bool();
|
||||||
|
case STRING:
|
||||||
|
return SchemaBuilder.string();
|
||||||
|
default:
|
||||||
|
throw new DataException("Unexpected type in Cast transformation: " + type);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Object castValueToType(Object value, Schema.Type targetType) {
|
||||||
|
try {
|
||||||
|
if (value == null) return null;
|
||||||
|
|
||||||
|
Schema.Type inferredType = ConnectSchema.schemaType(value.getClass());
|
||||||
|
if (inferredType == null) {
|
||||||
|
throw new DataException("Cast transformation was passed a value of type " + value.getClass()
|
||||||
|
+ " which is not supported by Connect's data API");
|
||||||
|
}
|
||||||
|
// Ensure the type we are trying to cast from is supported
|
||||||
|
validCastType(inferredType, FieldType.INPUT);
|
||||||
|
|
||||||
|
switch (targetType) {
|
||||||
|
case INT8:
|
||||||
|
return castToInt8(value);
|
||||||
|
case INT16:
|
||||||
|
return castToInt16(value);
|
||||||
|
case INT32:
|
||||||
|
return castToInt32(value);
|
||||||
|
case INT64:
|
||||||
|
return castToInt64(value);
|
||||||
|
case FLOAT32:
|
||||||
|
return castToFloat32(value);
|
||||||
|
case FLOAT64:
|
||||||
|
return castToFloat64(value);
|
||||||
|
case BOOLEAN:
|
||||||
|
return castToBoolean(value);
|
||||||
|
case STRING:
|
||||||
|
return castToString(value);
|
||||||
|
default:
|
||||||
|
throw new DataException(targetType.toString() + " is not supported in the Cast transformation.");
|
||||||
|
}
|
||||||
|
} catch (NumberFormatException e) {
|
||||||
|
throw new DataException("Value (" + value.toString() + ") was out of range for requested data type", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static byte castToInt8(Object value) {
|
||||||
|
if (value instanceof Number)
|
||||||
|
return ((Number) value).byteValue();
|
||||||
|
else if (value instanceof Boolean)
|
||||||
|
return ((boolean) value) ? (byte) 1 : (byte) 0;
|
||||||
|
else if (value instanceof String)
|
||||||
|
return Byte.parseByte((String) value);
|
||||||
|
else
|
||||||
|
throw new DataException("Unexpected type in Cast transformation: " + value.getClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static short castToInt16(Object value) {
|
||||||
|
if (value instanceof Number)
|
||||||
|
return ((Number) value).shortValue();
|
||||||
|
else if (value instanceof Boolean)
|
||||||
|
return ((boolean) value) ? (short) 1 : (short) 0;
|
||||||
|
else if (value instanceof String)
|
||||||
|
return Short.parseShort((String) value);
|
||||||
|
else
|
||||||
|
throw new DataException("Unexpected type in Cast transformation: " + value.getClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int castToInt32(Object value) {
|
||||||
|
if (value instanceof Number)
|
||||||
|
return ((Number) value).intValue();
|
||||||
|
else if (value instanceof Boolean)
|
||||||
|
return ((boolean) value) ? 1 : 0;
|
||||||
|
else if (value instanceof String)
|
||||||
|
return Integer.parseInt((String) value);
|
||||||
|
else
|
||||||
|
throw new DataException("Unexpected type in Cast transformation: " + value.getClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static long castToInt64(Object value) {
|
||||||
|
if (value instanceof Number)
|
||||||
|
return ((Number) value).longValue();
|
||||||
|
else if (value instanceof Boolean)
|
||||||
|
return ((boolean) value) ? (long) 1 : (long) 0;
|
||||||
|
else if (value instanceof String)
|
||||||
|
return Long.parseLong((String) value);
|
||||||
|
else
|
||||||
|
throw new DataException("Unexpected type in Cast transformation: " + value.getClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static float castToFloat32(Object value) {
|
||||||
|
if (value instanceof Number)
|
||||||
|
return ((Number) value).floatValue();
|
||||||
|
else if (value instanceof Boolean)
|
||||||
|
return ((boolean) value) ? 1.f : 0.f;
|
||||||
|
else if (value instanceof String)
|
||||||
|
return Float.parseFloat((String) value);
|
||||||
|
else
|
||||||
|
throw new DataException("Unexpected type in Cast transformation: " + value.getClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static double castToFloat64(Object value) {
|
||||||
|
if (value instanceof Number)
|
||||||
|
return ((Number) value).doubleValue();
|
||||||
|
else if (value instanceof Boolean)
|
||||||
|
return ((boolean) value) ? 1. : 0.;
|
||||||
|
else if (value instanceof String)
|
||||||
|
return Double.parseDouble((String) value);
|
||||||
|
else
|
||||||
|
throw new DataException("Unexpected type in Cast transformation: " + value.getClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static boolean castToBoolean(Object value) {
|
||||||
|
if (value instanceof Number)
|
||||||
|
return ((Number) value).longValue() != 0L;
|
||||||
|
else if (value instanceof Boolean)
|
||||||
|
return (Boolean) value;
|
||||||
|
else if (value instanceof String)
|
||||||
|
return Boolean.parseBoolean((String) value);
|
||||||
|
else
|
||||||
|
throw new DataException("Unexpected type in Cast transformation: " + value.getClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String castToString(Object value) {
|
||||||
|
return value.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract Schema operatingSchema(R record);
|
||||||
|
|
||||||
|
protected abstract Object operatingValue(R record);
|
||||||
|
|
||||||
|
protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue);
|
||||||
|
|
||||||
|
private static Map<String, Schema.Type> parseFieldTypes(List<String> mappings) {
|
||||||
|
final Map<String, Schema.Type> m = new HashMap<>();
|
||||||
|
boolean isWholeValueCast = false;
|
||||||
|
for (String mapping : mappings) {
|
||||||
|
final String[] parts = mapping.split(":");
|
||||||
|
if (parts.length > 2) {
|
||||||
|
throw new ConfigException(ReplaceField.ConfigName.RENAME, mappings, "Invalid rename mapping: " + mapping);
|
||||||
|
}
|
||||||
|
if (parts.length == 1) {
|
||||||
|
Schema.Type targetType = Schema.Type.valueOf(parts[0].trim().toUpperCase(Locale.ROOT));
|
||||||
|
m.put(WHOLE_VALUE_CAST, validCastType(targetType, FieldType.OUTPUT));
|
||||||
|
isWholeValueCast = true;
|
||||||
|
} else {
|
||||||
|
Schema.Type type;
|
||||||
|
try {
|
||||||
|
type = Schema.Type.valueOf(parts[1].trim().toUpperCase(Locale.ROOT));
|
||||||
|
} catch (IllegalArgumentException e) {
|
||||||
|
throw new ConfigException("Invalid type found in casting spec: " + parts[1].trim(), e);
|
||||||
|
}
|
||||||
|
m.put(parts[0].trim(), validCastType(type, FieldType.OUTPUT));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (isWholeValueCast && mappings.size() > 1) {
|
||||||
|
throw new ConfigException("Cast transformations that specify a type to cast the entire value to "
|
||||||
|
+ "may ony specify a single cast in their spec");
|
||||||
|
}
|
||||||
|
return m;
|
||||||
|
}
|
||||||
|
|
||||||
|
private enum FieldType {
|
||||||
|
INPUT, OUTPUT
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Schema.Type validCastType(Schema.Type type, FieldType fieldType) {
|
||||||
|
if (!SUPPORTED_CAST_TYPES.contains(type)) {
|
||||||
|
String message = "Cast transformation does not support casting to/from " + type
|
||||||
|
+ "; supported types are " + SUPPORTED_CAST_TYPES;
|
||||||
|
switch (fieldType) {
|
||||||
|
case INPUT:
|
||||||
|
throw new DataException(message);
|
||||||
|
case OUTPUT:
|
||||||
|
throw new ConfigException(message);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return type;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static final class Key<R extends ConnectRecord<R>> extends Cast<R> {
|
||||||
|
@Override
|
||||||
|
protected Schema operatingSchema(R record) {
|
||||||
|
return record.keySchema();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Object operatingValue(R record) {
|
||||||
|
return record.key();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
|
||||||
|
return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static final class Value<R extends ConnectRecord<R>> extends Cast<R> {
|
||||||
|
@Override
|
||||||
|
protected Schema operatingSchema(R record) {
|
||||||
|
return record.valueSchema();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Object operatingValue(R record) {
|
||||||
|
return record.value();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
|
||||||
|
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,281 @@
|
||||||
|
/*
|
||||||
|
* 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.transforms;
|
||||||
|
|
||||||
|
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.connect.connector.ConnectRecord;
|
||||||
|
import org.apache.kafka.connect.data.ConnectSchema;
|
||||||
|
import org.apache.kafka.connect.data.Field;
|
||||||
|
import org.apache.kafka.connect.data.Schema;
|
||||||
|
import org.apache.kafka.connect.data.SchemaBuilder;
|
||||||
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
import org.apache.kafka.connect.errors.DataException;
|
||||||
|
import org.apache.kafka.connect.transforms.util.SchemaUtil;
|
||||||
|
import org.apache.kafka.connect.transforms.util.SimpleConfig;
|
||||||
|
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.apache.kafka.connect.transforms.util.Requirements.requireMap;
|
||||||
|
import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct;
|
||||||
|
|
||||||
|
public abstract class Flatten<R extends ConnectRecord<R>> implements Transformation<R> {
|
||||||
|
|
||||||
|
public static final String OVERVIEW_DOC =
|
||||||
|
"Flatten a nested data structure, generating names for each field by concatenating the field names at each "
|
||||||
|
+ "level with a configurable delimiter character. Applies to Struct when schema present, or a Map "
|
||||||
|
+ "in the case of schemaless data. The default delimiter is '.'."
|
||||||
|
+ "<p/>Use the concrete transformation type designed for the record key (<code>" + Key.class.getName() + "</code>) "
|
||||||
|
+ "or value (<code>" + Value.class.getName() + "</code>).";
|
||||||
|
|
||||||
|
private static final String DELIMITER_CONFIG = "delimiter";
|
||||||
|
private static final String DELIMITER_DEFAULT = ".";
|
||||||
|
|
||||||
|
public static final ConfigDef CONFIG_DEF = new ConfigDef()
|
||||||
|
.define(DELIMITER_CONFIG, ConfigDef.Type.STRING, DELIMITER_DEFAULT, ConfigDef.Importance.MEDIUM,
|
||||||
|
"Delimiter to insert between field names from the input record when generating field names for the "
|
||||||
|
+ "output record");
|
||||||
|
|
||||||
|
private static final String PURPOSE = "flattening";
|
||||||
|
|
||||||
|
private String delimiter;
|
||||||
|
|
||||||
|
private Cache<Schema, Schema> schemaUpdateCache;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void configure(Map<String, ?> props) {
|
||||||
|
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
|
||||||
|
delimiter = config.getString(DELIMITER_CONFIG);
|
||||||
|
schemaUpdateCache = new SynchronizedCache<>(new LRUCache<Schema, Schema>(16));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public R apply(R record) {
|
||||||
|
if (operatingSchema(record) == null) {
|
||||||
|
return applySchemaless(record);
|
||||||
|
} else {
|
||||||
|
return applyWithSchema(record);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ConfigDef config() {
|
||||||
|
return CONFIG_DEF;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract Schema operatingSchema(R record);
|
||||||
|
|
||||||
|
protected abstract Object operatingValue(R record);
|
||||||
|
|
||||||
|
protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue);
|
||||||
|
|
||||||
|
private R applySchemaless(R record) {
|
||||||
|
final Map<String, Object> value = requireMap(operatingValue(record), PURPOSE);
|
||||||
|
final Map<String, Object> newValue = new LinkedHashMap<>();
|
||||||
|
applySchemaless(value, "", newValue);
|
||||||
|
return newRecord(record, null, newValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void applySchemaless(Map<String, Object> originalRecord, String fieldNamePrefix, Map<String, Object> newRecord) {
|
||||||
|
for (Map.Entry<String, Object> entry : originalRecord.entrySet()) {
|
||||||
|
final String fieldName = fieldName(fieldNamePrefix, entry.getKey());
|
||||||
|
Object value = entry.getValue();
|
||||||
|
if (value == null) {
|
||||||
|
newRecord.put(fieldName(fieldNamePrefix, entry.getKey()), null);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
Schema.Type inferredType = ConnectSchema.schemaType(value.getClass());
|
||||||
|
if (inferredType == null) {
|
||||||
|
throw new DataException("Flatten transformation was passed a value of type " + value.getClass()
|
||||||
|
+ " which is not supported by Connect's data API");
|
||||||
|
}
|
||||||
|
switch (inferredType) {
|
||||||
|
case INT8:
|
||||||
|
case INT16:
|
||||||
|
case INT32:
|
||||||
|
case INT64:
|
||||||
|
case FLOAT32:
|
||||||
|
case FLOAT64:
|
||||||
|
case BOOLEAN:
|
||||||
|
case STRING:
|
||||||
|
case BYTES:
|
||||||
|
newRecord.put(fieldName(fieldNamePrefix, entry.getKey()), entry.getValue());
|
||||||
|
break;
|
||||||
|
case MAP:
|
||||||
|
final Map<String, Object> fieldValue = requireMap(entry.getValue(), PURPOSE);
|
||||||
|
applySchemaless(fieldValue, fieldName, newRecord);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new DataException("Flatten transformation does not support " + entry.getValue().getClass()
|
||||||
|
+ " for record without schemas (for field " + fieldName + ").");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private R applyWithSchema(R record) {
|
||||||
|
final Struct value = requireStruct(operatingValue(record), PURPOSE);
|
||||||
|
|
||||||
|
Schema updatedSchema = schemaUpdateCache.get(value.schema());
|
||||||
|
if (updatedSchema == null) {
|
||||||
|
final SchemaBuilder builder = SchemaUtil.copySchemaBasics(value.schema(), SchemaBuilder.struct());
|
||||||
|
Struct defaultValue = (Struct) value.schema().defaultValue();
|
||||||
|
buildUpdatedSchema(value.schema(), "", builder, value.schema().isOptional(), defaultValue);
|
||||||
|
updatedSchema = builder.build();
|
||||||
|
schemaUpdateCache.put(value.schema(), updatedSchema);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Struct updatedValue = new Struct(updatedSchema);
|
||||||
|
buildWithSchema(value, "", updatedValue);
|
||||||
|
return newRecord(record, updatedSchema, updatedValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build an updated Struct Schema which flattens all nested fields into a single struct, handling cases where
|
||||||
|
* optionality and default values of the flattened fields are affected by the optionality and default values of
|
||||||
|
* parent/ancestor schemas (e.g. flattened field is optional because the parent schema was optional, even if the
|
||||||
|
* schema itself is marked as required).
|
||||||
|
* @param schema the schema to translate
|
||||||
|
* @param fieldNamePrefix the prefix to use on field names, i.e. the delimiter-joined set of ancestor field names
|
||||||
|
* @param newSchema the flattened schema being built
|
||||||
|
* @param optional true if any ancestor schema is optional
|
||||||
|
* @param defaultFromParent the default value, if any, included via the parent/ancestor schemas
|
||||||
|
*/
|
||||||
|
private void buildUpdatedSchema(Schema schema, String fieldNamePrefix, SchemaBuilder newSchema, boolean optional, Struct defaultFromParent) {
|
||||||
|
for (Field field : schema.fields()) {
|
||||||
|
final String fieldName = fieldName(fieldNamePrefix, field.name());
|
||||||
|
final boolean fieldIsOptional = optional || field.schema().isOptional();
|
||||||
|
Object fieldDefaultValue = null;
|
||||||
|
if (field.schema().defaultValue() != null) {
|
||||||
|
fieldDefaultValue = field.schema().defaultValue();
|
||||||
|
} else if (defaultFromParent != null) {
|
||||||
|
fieldDefaultValue = defaultFromParent.get(field);
|
||||||
|
}
|
||||||
|
switch (field.schema().type()) {
|
||||||
|
case INT8:
|
||||||
|
case INT16:
|
||||||
|
case INT32:
|
||||||
|
case INT64:
|
||||||
|
case FLOAT32:
|
||||||
|
case FLOAT64:
|
||||||
|
case BOOLEAN:
|
||||||
|
case STRING:
|
||||||
|
case BYTES:
|
||||||
|
newSchema.field(fieldName, convertFieldSchema(field.schema(), fieldIsOptional, fieldDefaultValue));
|
||||||
|
break;
|
||||||
|
case STRUCT:
|
||||||
|
buildUpdatedSchema(field.schema(), fieldName, newSchema, fieldIsOptional, (Struct) fieldDefaultValue);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new DataException("Flatten transformation does not support " + field.schema().type()
|
||||||
|
+ " for record without schemas (for field " + fieldName + ").");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert the schema for a field of a Struct with a primitive schema to the schema to be used for the flattened
|
||||||
|
* version, taking into account that we may need to override optionality and default values in the flattened version
|
||||||
|
* to take into account the optionality and default values of parent/ancestor schemas
|
||||||
|
* @param orig the original schema for the field
|
||||||
|
* @param optional whether the new flattened field should be optional
|
||||||
|
* @param defaultFromParent the default value either taken from the existing field or provided by the parent
|
||||||
|
*/
|
||||||
|
private Schema convertFieldSchema(Schema orig, boolean optional, Object defaultFromParent) {
|
||||||
|
// Note that we don't use the schema translation cache here. It might save us a bit of effort, but we really
|
||||||
|
// only care about caching top-level schema translations.
|
||||||
|
|
||||||
|
final SchemaBuilder builder = SchemaUtil.copySchemaBasics(orig);
|
||||||
|
if (optional)
|
||||||
|
builder.optional();
|
||||||
|
if (defaultFromParent != null)
|
||||||
|
builder.defaultValue(defaultFromParent);
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void buildWithSchema(Struct record, String fieldNamePrefix, Struct newRecord) {
|
||||||
|
for (Field field : record.schema().fields()) {
|
||||||
|
final String fieldName = fieldName(fieldNamePrefix, field.name());
|
||||||
|
switch (field.schema().type()) {
|
||||||
|
case INT8:
|
||||||
|
case INT16:
|
||||||
|
case INT32:
|
||||||
|
case INT64:
|
||||||
|
case FLOAT32:
|
||||||
|
case FLOAT64:
|
||||||
|
case BOOLEAN:
|
||||||
|
case STRING:
|
||||||
|
case BYTES:
|
||||||
|
newRecord.put(fieldName, record.get(field));
|
||||||
|
break;
|
||||||
|
case STRUCT:
|
||||||
|
buildWithSchema(record.getStruct(field.name()), fieldName, newRecord);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new DataException("Flatten transformation does not support " + field.schema().type()
|
||||||
|
+ " for record without schemas (for field " + fieldName + ").");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private String fieldName(String prefix, String fieldName) {
|
||||||
|
return prefix.isEmpty() ? fieldName : (prefix + delimiter + fieldName);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Key<R extends ConnectRecord<R>> extends Flatten<R> {
|
||||||
|
@Override
|
||||||
|
protected Schema operatingSchema(R record) {
|
||||||
|
return record.keySchema();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Object operatingValue(R record) {
|
||||||
|
return record.key();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
|
||||||
|
return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Value<R extends ConnectRecord<R>> extends Flatten<R> {
|
||||||
|
@Override
|
||||||
|
protected Schema operatingSchema(R record) {
|
||||||
|
return record.valueSchema();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Object operatingValue(R record) {
|
||||||
|
return record.value();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
|
||||||
|
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -32,6 +32,7 @@ public class Requirements {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
public static Map<String, Object> requireMap(Object value, String purpose) {
|
public static Map<String, Object> requireMap(Object value, String purpose) {
|
||||||
if (!(value instanceof Map)) {
|
if (!(value instanceof Map)) {
|
||||||
throw new DataException("Only Map objects supported in absence of schema for [" + purpose + "], found: " + nullSafeClassName(value));
|
throw new DataException("Only Map objects supported in absence of schema for [" + purpose + "], found: " + nullSafeClassName(value));
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,10 @@ import java.util.Map;
|
||||||
|
|
||||||
public class SchemaUtil {
|
public class SchemaUtil {
|
||||||
|
|
||||||
|
public static SchemaBuilder copySchemaBasics(Schema source) {
|
||||||
|
return copySchemaBasics(source, new SchemaBuilder(source.type()));
|
||||||
|
}
|
||||||
|
|
||||||
public static SchemaBuilder copySchemaBasics(Schema source, SchemaBuilder builder) {
|
public static SchemaBuilder copySchemaBasics(Schema source, SchemaBuilder builder) {
|
||||||
builder.name(source.name());
|
builder.name(source.name());
|
||||||
builder.version(source.version());
|
builder.version(source.version());
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,384 @@
|
||||||
|
/*
|
||||||
|
* 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.transforms;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.config.ConfigException;
|
||||||
|
import org.apache.kafka.connect.data.Schema;
|
||||||
|
import org.apache.kafka.connect.data.SchemaBuilder;
|
||||||
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
import org.apache.kafka.connect.errors.DataException;
|
||||||
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNull;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
public class CastTest {
|
||||||
|
|
||||||
|
@Test(expected = ConfigException.class)
|
||||||
|
public void testConfigEmpty() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Key<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, ""));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = ConfigException.class)
|
||||||
|
public void testConfigInvalidSchemaType() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Key<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:faketype"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = ConfigException.class)
|
||||||
|
public void testConfigInvalidTargetType() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Key<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:array"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = ConfigException.class)
|
||||||
|
public void testConfigInvalidMap() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Key<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int8:extra"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = ConfigException.class)
|
||||||
|
public void testConfigMixWholeAndFieldTransformation() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Key<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int8,int32"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordKeyWithSchema() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Key<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42, Schema.STRING_SCHEMA, "bogus"));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.INT8, transformed.keySchema().type());
|
||||||
|
assertEquals((byte) 42, transformed.key());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaInt8() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.INT8, transformed.valueSchema().type());
|
||||||
|
assertEquals((byte) 42, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaInt16() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int16"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.INT16, transformed.valueSchema().type());
|
||||||
|
assertEquals((short) 42, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaInt32() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int32"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.INT32, transformed.valueSchema().type());
|
||||||
|
assertEquals(42, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaInt64() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int64"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.INT64, transformed.valueSchema().type());
|
||||||
|
assertEquals((long) 42, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaFloat32() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "float32"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.FLOAT32, transformed.valueSchema().type());
|
||||||
|
assertEquals(42.f, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaFloat64() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "float64"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.FLOAT64, transformed.valueSchema().type());
|
||||||
|
assertEquals(42., transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaBooleanTrue() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "boolean"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.BOOLEAN, transformed.valueSchema().type());
|
||||||
|
assertEquals(true, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaBooleanFalse() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "boolean"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 0));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.BOOLEAN, transformed.valueSchema().type());
|
||||||
|
assertEquals(false, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueWithSchemaString() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "string"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
Schema.INT32_SCHEMA, 42));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.STRING, transformed.valueSchema().type());
|
||||||
|
assertEquals("42", transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordDefaultValue() {
|
||||||
|
// Validate default value in schema is correctly converted
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int32"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
SchemaBuilder.float32().defaultValue(-42.125f).build(), 42.125f));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.INT32, transformed.valueSchema().type());
|
||||||
|
assertEquals(42, transformed.value());
|
||||||
|
assertEquals(-42, transformed.valueSchema().defaultValue());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordKeySchemaless() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Key<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42, Schema.STRING_SCHEMA, "bogus"));
|
||||||
|
|
||||||
|
assertNull(transformed.keySchema());
|
||||||
|
assertEquals((byte) 42, transformed.key());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessInt8() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals((byte) 42, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessInt16() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int16"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals((short) 42, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessInt32() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int32"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals(42, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessInt64() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int64"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals((long) 42, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessFloat32() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "float32"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals(42.f, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessFloat64() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "float64"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals(42., transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessBooleanTrue() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "boolean"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals(true, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessBooleanFalse() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "boolean"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 0));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals(false, transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castWholeRecordValueSchemalessString() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "string"));
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, 42));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals("42", transformed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = DataException.class)
|
||||||
|
public void castWholeRecordValueSchemalessUnsupportedType() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
|
||||||
|
xform.apply(new SourceRecord(null, null, "topic", 0, null, Collections.singletonList("foo")));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void castFieldsWithSchema() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8:int16,int16:int32,int32:int64,int64:boolean,float32:float64,float64:boolean,boolean:int8,string:int32,optional:int32"));
|
||||||
|
|
||||||
|
// Include an optional fields and fields with defaults to validate their values are passed through properly
|
||||||
|
SchemaBuilder builder = SchemaBuilder.struct();
|
||||||
|
builder.field("int8", Schema.INT8_SCHEMA);
|
||||||
|
builder.field("int16", Schema.OPTIONAL_INT16_SCHEMA);
|
||||||
|
builder.field("int32", SchemaBuilder.int32().defaultValue(2).build());
|
||||||
|
builder.field("int64", Schema.INT64_SCHEMA);
|
||||||
|
builder.field("float32", Schema.FLOAT32_SCHEMA);
|
||||||
|
// Default value here ensures we correctly convert default values
|
||||||
|
builder.field("float64", SchemaBuilder.float64().defaultValue(-1.125).build());
|
||||||
|
builder.field("boolean", Schema.BOOLEAN_SCHEMA);
|
||||||
|
builder.field("string", Schema.STRING_SCHEMA);
|
||||||
|
builder.field("optional", Schema.OPTIONAL_FLOAT32_SCHEMA);
|
||||||
|
Schema supportedTypesSchema = builder.build();
|
||||||
|
|
||||||
|
Struct recordValue = new Struct(supportedTypesSchema);
|
||||||
|
recordValue.put("int8", (byte) 8);
|
||||||
|
recordValue.put("int16", (short) 16);
|
||||||
|
recordValue.put("int32", 32);
|
||||||
|
recordValue.put("int64", (long) 64);
|
||||||
|
recordValue.put("float32", 32.f);
|
||||||
|
recordValue.put("float64", -64.);
|
||||||
|
recordValue.put("boolean", true);
|
||||||
|
recordValue.put("string", "42");
|
||||||
|
// optional field intentionally omitted
|
||||||
|
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
supportedTypesSchema, recordValue));
|
||||||
|
|
||||||
|
assertEquals((short) 8, ((Struct) transformed.value()).get("int8"));
|
||||||
|
assertTrue(((Struct) transformed.value()).schema().field("int16").schema().isOptional());
|
||||||
|
assertEquals(16, ((Struct) transformed.value()).get("int16"));
|
||||||
|
assertEquals((long) 32, ((Struct) transformed.value()).get("int32"));
|
||||||
|
assertEquals(2L, ((Struct) transformed.value()).schema().field("int32").schema().defaultValue());
|
||||||
|
assertEquals(true, ((Struct) transformed.value()).get("int64"));
|
||||||
|
assertEquals(32., ((Struct) transformed.value()).get("float32"));
|
||||||
|
assertEquals(true, ((Struct) transformed.value()).get("float64"));
|
||||||
|
assertEquals(true, ((Struct) transformed.value()).schema().field("float64").schema().defaultValue());
|
||||||
|
assertEquals((byte) 1, ((Struct) transformed.value()).get("boolean"));
|
||||||
|
assertEquals(42, ((Struct) transformed.value()).get("string"));
|
||||||
|
assertNull(((Struct) transformed.value()).get("optional"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Test
|
||||||
|
public void castFieldsSchemaless() {
|
||||||
|
final Cast<SourceRecord> xform = new Cast.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8:int16,int16:int32,int32:int64,int64:boolean,float32:float64,float64:boolean,boolean:int8,string:int32"));
|
||||||
|
Map<String, Object> recordValue = new HashMap<>();
|
||||||
|
recordValue.put("int8", (byte) 8);
|
||||||
|
recordValue.put("int16", (short) 16);
|
||||||
|
recordValue.put("int32", 32);
|
||||||
|
recordValue.put("int64", (long) 64);
|
||||||
|
recordValue.put("float32", 32.f);
|
||||||
|
recordValue.put("float64", -64.);
|
||||||
|
recordValue.put("boolean", true);
|
||||||
|
recordValue.put("string", "42");
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0,
|
||||||
|
null, recordValue));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertEquals((short) 8, ((Map<String, Object>) transformed.value()).get("int8"));
|
||||||
|
assertEquals(16, ((Map<String, Object>) transformed.value()).get("int16"));
|
||||||
|
assertEquals((long) 32, ((Map<String, Object>) transformed.value()).get("int32"));
|
||||||
|
assertEquals(true, ((Map<String, Object>) transformed.value()).get("int64"));
|
||||||
|
assertEquals(32., ((Map<String, Object>) transformed.value()).get("float32"));
|
||||||
|
assertEquals(true, ((Map<String, Object>) transformed.value()).get("float64"));
|
||||||
|
assertEquals((byte) 1, ((Map<String, Object>) transformed.value()).get("boolean"));
|
||||||
|
assertEquals(42, ((Map<String, Object>) transformed.value()).get("string"));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,257 @@
|
||||||
|
/*
|
||||||
|
* 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.transforms;
|
||||||
|
|
||||||
|
import org.apache.kafka.connect.data.Schema;
|
||||||
|
import org.apache.kafka.connect.data.SchemaBuilder;
|
||||||
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
import org.apache.kafka.connect.errors.DataException;
|
||||||
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
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.assertTrue;
|
||||||
|
|
||||||
|
public class FlattenTest {
|
||||||
|
|
||||||
|
@Test(expected = DataException.class)
|
||||||
|
public void topLevelStructRequired() {
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Value<>();
|
||||||
|
xform.configure(Collections.<String, String>emptyMap());
|
||||||
|
xform.apply(new SourceRecord(null, null, "topic", 0, Schema.INT32_SCHEMA, 42));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = DataException.class)
|
||||||
|
public void topLevelMapRequired() {
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Value<>();
|
||||||
|
xform.configure(Collections.<String, String>emptyMap());
|
||||||
|
xform.apply(new SourceRecord(null, null, "topic", 0, null, 42));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNestedStruct() {
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Value<>();
|
||||||
|
xform.configure(Collections.<String, String>emptyMap());
|
||||||
|
|
||||||
|
SchemaBuilder builder = SchemaBuilder.struct();
|
||||||
|
builder.field("int8", Schema.INT8_SCHEMA);
|
||||||
|
builder.field("int16", Schema.INT16_SCHEMA);
|
||||||
|
builder.field("int32", Schema.INT32_SCHEMA);
|
||||||
|
builder.field("int64", Schema.INT64_SCHEMA);
|
||||||
|
builder.field("float32", Schema.FLOAT32_SCHEMA);
|
||||||
|
builder.field("float64", Schema.FLOAT64_SCHEMA);
|
||||||
|
builder.field("boolean", Schema.BOOLEAN_SCHEMA);
|
||||||
|
builder.field("string", Schema.STRING_SCHEMA);
|
||||||
|
builder.field("bytes", Schema.BYTES_SCHEMA);
|
||||||
|
Schema supportedTypesSchema = builder.build();
|
||||||
|
|
||||||
|
builder = SchemaBuilder.struct();
|
||||||
|
builder.field("B", supportedTypesSchema);
|
||||||
|
Schema oneLevelNestedSchema = builder.build();
|
||||||
|
|
||||||
|
builder = SchemaBuilder.struct();
|
||||||
|
builder.field("A", oneLevelNestedSchema);
|
||||||
|
Schema twoLevelNestedSchema = builder.build();
|
||||||
|
|
||||||
|
Struct supportedTypes = new Struct(supportedTypesSchema);
|
||||||
|
supportedTypes.put("int8", (byte) 8);
|
||||||
|
supportedTypes.put("int16", (short) 16);
|
||||||
|
supportedTypes.put("int32", 32);
|
||||||
|
supportedTypes.put("int64", (long) 64);
|
||||||
|
supportedTypes.put("float32", 32.f);
|
||||||
|
supportedTypes.put("float64", 64.);
|
||||||
|
supportedTypes.put("boolean", true);
|
||||||
|
supportedTypes.put("string", "stringy");
|
||||||
|
supportedTypes.put("bytes", "bytes".getBytes());
|
||||||
|
|
||||||
|
Struct oneLevelNestedStruct = new Struct(oneLevelNestedSchema);
|
||||||
|
oneLevelNestedStruct.put("B", supportedTypes);
|
||||||
|
|
||||||
|
Struct twoLevelNestedStruct = new Struct(twoLevelNestedSchema);
|
||||||
|
twoLevelNestedStruct.put("A", oneLevelNestedStruct);
|
||||||
|
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null,
|
||||||
|
"topic", 0,
|
||||||
|
twoLevelNestedSchema, twoLevelNestedStruct));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.STRUCT, transformed.valueSchema().type());
|
||||||
|
Struct transformedStruct = (Struct) transformed.value();
|
||||||
|
assertEquals(9, transformedStruct.schema().fields().size());
|
||||||
|
assertEquals(8, (byte) transformedStruct.getInt8("A.B.int8"));
|
||||||
|
assertEquals(16, (short) transformedStruct.getInt16("A.B.int16"));
|
||||||
|
assertEquals(32, (int) transformedStruct.getInt32("A.B.int32"));
|
||||||
|
assertEquals(64L, (long) transformedStruct.getInt64("A.B.int64"));
|
||||||
|
assertEquals(32.f, transformedStruct.getFloat32("A.B.float32"), 0.f);
|
||||||
|
assertEquals(64., transformedStruct.getFloat64("A.B.float64"), 0.);
|
||||||
|
assertEquals(true, transformedStruct.getBoolean("A.B.boolean"));
|
||||||
|
assertEquals("stringy", transformedStruct.getString("A.B.string"));
|
||||||
|
assertArrayEquals("bytes".getBytes(), transformedStruct.getBytes("A.B.bytes"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNestedMapWithDelimiter() {
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Value<>();
|
||||||
|
xform.configure(Collections.singletonMap("delimiter", "#"));
|
||||||
|
|
||||||
|
Map<String, Object> supportedTypes = new HashMap<>();
|
||||||
|
supportedTypes.put("int8", (byte) 8);
|
||||||
|
supportedTypes.put("int16", (short) 16);
|
||||||
|
supportedTypes.put("int32", 32);
|
||||||
|
supportedTypes.put("int64", (long) 64);
|
||||||
|
supportedTypes.put("float32", 32.f);
|
||||||
|
supportedTypes.put("float64", 64.);
|
||||||
|
supportedTypes.put("boolean", true);
|
||||||
|
supportedTypes.put("string", "stringy");
|
||||||
|
supportedTypes.put("bytes", "bytes".getBytes());
|
||||||
|
|
||||||
|
Map<String, Object> oneLevelNestedMap = Collections.singletonMap("B", (Object) supportedTypes);
|
||||||
|
Map<String, Object> twoLevelNestedMap = Collections.singletonMap("A", (Object) oneLevelNestedMap);
|
||||||
|
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null,
|
||||||
|
"topic", 0,
|
||||||
|
null, twoLevelNestedMap));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertTrue(transformed.value() instanceof Map);
|
||||||
|
Map<String, Object> transformedMap = (Map<String, Object>) transformed.value();
|
||||||
|
assertEquals(9, transformedMap.size());
|
||||||
|
assertEquals((byte) 8, transformedMap.get("A#B#int8"));
|
||||||
|
assertEquals((short) 16, transformedMap.get("A#B#int16"));
|
||||||
|
assertEquals(32, transformedMap.get("A#B#int32"));
|
||||||
|
assertEquals((long) 64, transformedMap.get("A#B#int64"));
|
||||||
|
assertEquals(32.f, (float) transformedMap.get("A#B#float32"), 0.f);
|
||||||
|
assertEquals(64., (double) transformedMap.get("A#B#float64"), 0.);
|
||||||
|
assertEquals(true, transformedMap.get("A#B#boolean"));
|
||||||
|
assertEquals("stringy", transformedMap.get("A#B#string"));
|
||||||
|
assertArrayEquals("bytes".getBytes(), (byte[]) transformedMap.get("A#B#bytes"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOptionalFieldStruct() {
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Value<>();
|
||||||
|
xform.configure(Collections.<String, String>emptyMap());
|
||||||
|
|
||||||
|
SchemaBuilder builder = SchemaBuilder.struct();
|
||||||
|
builder.field("opt_int32", Schema.OPTIONAL_INT32_SCHEMA);
|
||||||
|
Schema supportedTypesSchema = builder.build();
|
||||||
|
|
||||||
|
builder = SchemaBuilder.struct();
|
||||||
|
builder.field("B", supportedTypesSchema);
|
||||||
|
Schema oneLevelNestedSchema = builder.build();
|
||||||
|
|
||||||
|
Struct supportedTypes = new Struct(supportedTypesSchema);
|
||||||
|
supportedTypes.put("opt_int32", null);
|
||||||
|
|
||||||
|
Struct oneLevelNestedStruct = new Struct(oneLevelNestedSchema);
|
||||||
|
oneLevelNestedStruct.put("B", supportedTypes);
|
||||||
|
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null,
|
||||||
|
"topic", 0,
|
||||||
|
oneLevelNestedSchema, oneLevelNestedStruct));
|
||||||
|
|
||||||
|
assertEquals(Schema.Type.STRUCT, transformed.valueSchema().type());
|
||||||
|
Struct transformedStruct = (Struct) transformed.value();
|
||||||
|
assertNull(transformedStruct.get("B.opt_int32"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOptionalFieldMap() {
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Value<>();
|
||||||
|
xform.configure(Collections.<String, String>emptyMap());
|
||||||
|
|
||||||
|
Map<String, Object> supportedTypes = new HashMap<>();
|
||||||
|
supportedTypes.put("opt_int32", null);
|
||||||
|
|
||||||
|
Map<String, Object> oneLevelNestedMap = Collections.singletonMap("B", (Object) supportedTypes);
|
||||||
|
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null,
|
||||||
|
"topic", 0,
|
||||||
|
null, oneLevelNestedMap));
|
||||||
|
|
||||||
|
assertNull(transformed.valueSchema());
|
||||||
|
assertTrue(transformed.value() instanceof Map);
|
||||||
|
Map<String, Object> transformedMap = (Map<String, Object>) transformed.value();
|
||||||
|
|
||||||
|
assertNull(transformedMap.get("B.opt_int32"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testKey() {
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Key<>();
|
||||||
|
xform.configure(Collections.<String, String>emptyMap());
|
||||||
|
|
||||||
|
Map<String, Map<String, Integer>> key = Collections.singletonMap("A", Collections.singletonMap("B", 12));
|
||||||
|
SourceRecord src = new SourceRecord(null, null, "topic", null, key, null, null);
|
||||||
|
SourceRecord transformed = xform.apply(src);
|
||||||
|
|
||||||
|
assertNull(transformed.keySchema());
|
||||||
|
assertTrue(transformed.key() instanceof Map);
|
||||||
|
Map<String, Object> transformedMap = (Map<String, Object>) transformed.key();
|
||||||
|
assertEquals(12, transformedMap.get("A.B"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = DataException.class)
|
||||||
|
public void testUnsupportedTypeInMap() {
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Value<>();
|
||||||
|
xform.configure(Collections.<String, String>emptyMap());
|
||||||
|
Object value = Collections.singletonMap("foo", Arrays.asList("bar", "baz"));
|
||||||
|
xform.apply(new SourceRecord(null, null, "topic", 0, null, value));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOptionalAndDefaultValuesNested() {
|
||||||
|
// If we have a nested structure where an entire sub-Struct is optional, all flattened fields generated from its
|
||||||
|
// children should also be optional. Similarly, if the parent Struct has a default value, the default value for
|
||||||
|
// the flattened field
|
||||||
|
|
||||||
|
final Flatten<SourceRecord> xform = new Flatten.Value<>();
|
||||||
|
xform.configure(Collections.<String, String>emptyMap());
|
||||||
|
|
||||||
|
SchemaBuilder builder = SchemaBuilder.struct().optional();
|
||||||
|
builder.field("req_field", Schema.STRING_SCHEMA);
|
||||||
|
builder.field("opt_field", SchemaBuilder.string().optional().defaultValue("child_default").build());
|
||||||
|
Struct childDefaultValue = new Struct(builder);
|
||||||
|
childDefaultValue.put("req_field", "req_default");
|
||||||
|
builder.defaultValue(childDefaultValue);
|
||||||
|
Schema schema = builder.build();
|
||||||
|
// Intentionally leave this entire value empty since it is optional
|
||||||
|
Struct value = new Struct(schema);
|
||||||
|
|
||||||
|
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0, schema, value));
|
||||||
|
|
||||||
|
assertNotNull(transformed);
|
||||||
|
Schema transformedSchema = transformed.valueSchema();
|
||||||
|
assertEquals(Schema.Type.STRUCT, transformedSchema.type());
|
||||||
|
assertEquals(2, transformedSchema.fields().size());
|
||||||
|
// Required field should pick up both being optional and the default value from the parent
|
||||||
|
Schema transformedReqFieldSchema = SchemaBuilder.string().optional().defaultValue("req_default").build();
|
||||||
|
assertEquals(transformedReqFieldSchema, transformedSchema.field("req_field").schema());
|
||||||
|
// The optional field should still be optional but should have picked up the default value. However, since
|
||||||
|
// the parent didn't specify the default explicitly, we should still be using the field's normal default
|
||||||
|
Schema transformedOptFieldSchema = SchemaBuilder.string().optional().defaultValue("child_default").build();
|
||||||
|
assertEquals(transformedOptFieldSchema, transformedSchema.field("opt_field").schema());
|
||||||
|
}
|
||||||
|
}
|
||||||
Loading…
Reference in New Issue