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. | ||||
|     private Map<String, String> parameters; | ||||
| 
 | ||||
|     private SchemaBuilder(Type type) { | ||||
|     public SchemaBuilder(Type type) { | ||||
|         this.type = type; | ||||
|         if (type == Type.STRUCT) { | ||||
|             fields = new LinkedHashMap<>(); | ||||
|  |  | |||
|  | @ -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) { | ||||
|  |  | |||
|  | @ -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) { | ||||
|         if (!(value instanceof Map)) { | ||||
|             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 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()); | ||||
|  |  | |||
|  | @ -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