Repository: kafka Updated Branches: refs/heads/trunk ebc7f7caa -> 1cea4d8f5
KAFKA-4714; Flatten and Cast single message transforms (KIP-66) Author: Ewen Cheslack-Postava <[email protected]> Reviewers: Konstantine Karantasis <[email protected]>, Shikhar Bhushan <[email protected]>, Jason Gustafson <[email protected]> Closes #2458 from ewencp/kafka-3209-even-more-transforms Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1cea4d8f Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1cea4d8f Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1cea4d8f Branch: refs/heads/trunk Commit: 1cea4d8f5a51cc5795ddd3af2ea015b9e14d937d Parents: ebc7f7c Author: Ewen Cheslack-Postava <[email protected]> Authored: Tue May 16 23:05:35 2017 -0700 Committer: Jason Gustafson <[email protected]> Committed: Tue May 16 23:05:35 2017 -0700 ---------------------------------------------------------------------- .../kafka/connect/data/SchemaBuilder.java | 2 +- .../kafka/connect/tools/TransformationDoc.java | 6 +- .../apache/kafka/connect/transforms/Cast.java | 417 +++++++++++++++++++ .../kafka/connect/transforms/Flatten.java | 281 +++++++++++++ .../connect/transforms/util/Requirements.java | 1 + .../connect/transforms/util/SchemaUtil.java | 4 + .../kafka/connect/transforms/CastTest.java | 384 +++++++++++++++++ .../kafka/connect/transforms/FlattenTest.java | 257 ++++++++++++ 8 files changed, 1350 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java ---------------------------------------------------------------------- diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java index f0c4586..058660e 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java @@ -75,7 +75,7 @@ public class SchemaBuilder implements Schema { // Additional parameters for logical types. private Map<String, String> parameters; - private SchemaBuilder(Type type) { + public SchemaBuilder(Type type) { this.type = type; if (type == Type.STRUCT) { fields = new LinkedHashMap<>(); http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java ---------------------------------------------------------------------- diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java index 5616613..1a8f0a8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -17,7 +17,9 @@ package org.apache.kafka.connect.tools; 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.Flatten; import org.apache.kafka.connect.transforms.HoistField; import org.apache.kafka.connect.transforms.InsertField; 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(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.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) { http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java ---------------------------------------------------------------------- diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java new file mode 100644 index 0000000..17be48c --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java @@ -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()); + } + } + +} http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Flatten.java ---------------------------------------------------------------------- diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Flatten.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Flatten.java new file mode 100644 index 0000000..c5e4000 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Flatten.java @@ -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()); + } + } + +} http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java ---------------------------------------------------------------------- diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java index 1d2df45..0220907 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java @@ -32,6 +32,7 @@ public class Requirements { } } + @SuppressWarnings("unchecked") public static Map<String, Object> requireMap(Object value, String purpose) { if (!(value instanceof Map)) { throw new DataException("Only Map objects supported in absence of schema for [" + purpose + "], found: " + nullSafeClassName(value)); http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java ---------------------------------------------------------------------- diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java index e2b7c98..3c497cb 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java @@ -23,6 +23,10 @@ import java.util.Map; public class SchemaUtil { + public static SchemaBuilder copySchemaBasics(Schema source) { + return copySchemaBasics(source, new SchemaBuilder(source.type())); + } + public static SchemaBuilder copySchemaBasics(Schema source, SchemaBuilder builder) { builder.name(source.name()); builder.version(source.version()); http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java ---------------------------------------------------------------------- diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java new file mode 100644 index 0000000..88afafc --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java @@ -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")); + } + +} http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java ---------------------------------------------------------------------- diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java new file mode 100644 index 0000000..86851f3 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java @@ -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()); + } +}
