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:
Ewen Cheslack-Postava 2017-05-16 23:05:35 -07:00 committed by Jason Gustafson
parent ebc7f7caae
commit 1cea4d8f5a
8 changed files with 1350 additions and 2 deletions

View File

@ -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<>();

View File

@ -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) {

View File

@ -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());
}
}
}

View File

@ -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());
}
}
}

View File

@ -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));

View File

@ -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());

View File

@ -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"));
}
}

View File

@ -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());
}
}