Add km module kafka

This commit is contained in:
leewei
2023-02-14 16:27:47 +08:00
parent 229140f067
commit 0b8160a714
4039 changed files with 718112 additions and 46204 deletions

View File

@@ -0,0 +1,463 @@
/*
* 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.Date;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Schema.Type;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.data.Time;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.data.Values;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.transforms.util.SchemaUtil;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.EnumSet;
import java.util.HashMap;
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> {
private static final Logger log = LoggerFactory.getLogger(Cast.class);
// 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_INPUT_TYPES = EnumSet.of(
Schema.Type.INT8, Schema.Type.INT16, Schema.Type.INT32, Schema.Type.INT64,
Schema.Type.FLOAT32, Schema.Type.FLOAT64, Schema.Type.BOOLEAN,
Schema.Type.STRING, Schema.Type.BYTES
);
private static final Set<Schema.Type> SUPPORTED_CAST_OUTPUT_TYPES = EnumSet.of(
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<>(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(null, 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(null, 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(valueSchema, 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(field.schema(), origFieldValue, targetType) : origFieldValue;
log.trace("Cast field '{}' from '{}' to '{}'", field.name(), origFieldValue, newFieldValue);
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()) {
if (casts.containsKey(field.name())) {
SchemaBuilder fieldBuilder = convertFieldType(casts.get(field.name()));
if (field.schema().isOptional())
fieldBuilder.optional();
if (field.schema().defaultValue() != null) {
Schema fieldSchema = field.schema();
fieldBuilder.defaultValue(castValueToType(fieldSchema, fieldSchema.defaultValue(), fieldBuilder.type()));
}
builder.field(field.name(), fieldBuilder.build());
} else {
builder.field(field.name(), field.schema());
}
}
}
if (valueSchema.isOptional())
builder.optional();
if (valueSchema.defaultValue() != null)
builder.defaultValue(castValueToType(valueSchema, 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 encodeLogicalType(Schema schema, Object value) {
switch (schema.name()) {
case Date.LOGICAL_NAME:
return Date.fromLogical(schema, (java.util.Date) value);
case Time.LOGICAL_NAME:
return Time.fromLogical(schema, (java.util.Date) value);
case Timestamp.LOGICAL_NAME:
return Timestamp.fromLogical(schema, (java.util.Date) value);
}
return value;
}
private static Object castValueToType(Schema schema, Object value, Schema.Type targetType) {
try {
if (value == null) return null;
Schema.Type inferredType = schema == null ? ConnectSchema.schemaType(value.getClass()) :
schema.type();
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);
// Perform logical type encoding to their internal representation.
if (schema != null && schema.name() != null && targetType != Type.STRING) {
value = encodeLogicalType(schema, value);
}
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) {
if (value instanceof java.util.Date) {
java.util.Date dateValue = (java.util.Date) value;
return Values.dateFormatFor(dateValue).format(dateValue);
} else {
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) {
switch (fieldType) {
case INPUT:
if (!SUPPORTED_CAST_INPUT_TYPES.contains(type)) {
throw new DataException("Cast transformation does not support casting from " +
type + "; supported types are " + SUPPORTED_CAST_INPUT_TYPES);
}
break;
case OUTPUT:
if (!SUPPORTED_CAST_OUTPUT_TYPES.contains(type)) {
throw new ConfigException("Cast transformation does not support casting to " +
type + "; supported types are " + SUPPORTED_CAST_OUTPUT_TYPES);
}
break;
}
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,121 @@
/*
* 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.ConfigDef;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.Map;
import static org.apache.kafka.connect.transforms.util.Requirements.requireMapOrNull;
import static org.apache.kafka.connect.transforms.util.Requirements.requireStructOrNull;
public abstract class ExtractField<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC =
"Extract the specified field from a Struct when schema present, or a Map in the case of schemaless data. "
+ "Any null values are passed through unmodified."
+ "<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 FIELD_CONFIG = "field";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(FIELD_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM, "Field name to extract.");
private static final String PURPOSE = "field extraction";
private String fieldName;
@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
fieldName = config.getString(FIELD_CONFIG);
}
@Override
public R apply(R record) {
final Schema schema = operatingSchema(record);
if (schema == null) {
final Map<String, Object> value = requireMapOrNull(operatingValue(record), PURPOSE);
return newRecord(record, null, value == null ? null : value.get(fieldName));
} else {
final Struct value = requireStructOrNull(operatingValue(record), PURPOSE);
Field field = schema.field(fieldName);
if (field == null) {
throw new IllegalArgumentException("Unknown field: " + fieldName);
}
return newRecord(record, field.schema(), value == null ? null : value.get(fieldName));
}
}
@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);
public static class Key<R extends ConnectRecord<R>> extends ExtractField<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 ExtractField<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,290 @@
/*
* 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.requireStructOrNull;
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<>(16));
}
@Override
public R apply(R record) {
if (operatingValue(record) == null) {
return record;
} else 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 = requireStructOrNull(operatingValue(record), PURPOSE);
Schema schema = operatingSchema(record);
Schema updatedSchema = schemaUpdateCache.get(schema);
if (updatedSchema == null) {
final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct());
Struct defaultValue = (Struct) schema.defaultValue();
buildUpdatedSchema(schema, "", builder, schema.isOptional(), defaultValue);
updatedSchema = builder.build();
schemaUpdateCache.put(schema, updatedSchema);
}
if (value == null) {
return newRecord(record, updatedSchema, null);
} else {
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 with 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) {
if (record == null) {
return;
}
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 with 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

@@ -0,0 +1,126 @@
/*
* 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.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.Collections;
import java.util.Map;
public abstract class HoistField<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC =
"Wrap data using the specified field name in a Struct when schema present, or a Map in the case of schemaless data."
+ "<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 FIELD_CONFIG = "field";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(FIELD_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM,
"Field name for the single field that will be created in the resulting Struct or Map.");
private Cache<Schema, Schema> schemaUpdateCache;
private String fieldName;
@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
fieldName = config.getString("field");
schemaUpdateCache = new SynchronizedCache<>(new LRUCache<>(16));
}
@Override
public R apply(R record) {
final Schema schema = operatingSchema(record);
final Object value = operatingValue(record);
if (schema == null) {
return newRecord(record, null, Collections.singletonMap(fieldName, value));
} else {
Schema updatedSchema = schemaUpdateCache.get(schema);
if (updatedSchema == null) {
updatedSchema = SchemaBuilder.struct().field(fieldName, schema).build();
schemaUpdateCache.put(schema, updatedSchema);
}
final Struct updatedValue = new Struct(updatedSchema).put(fieldName, value);
return newRecord(record, updatedSchema, updatedValue);
}
}
@Override
public void close() {
schemaUpdateCache = null;
}
@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);
public static class Key<R extends ConnectRecord<R>> extends HoistField<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 HoistField<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,277 @@
/*
* 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.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.data.Timestamp;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import org.apache.kafka.connect.transforms.util.SchemaUtil;
import java.util.Date;
import java.util.HashMap;
import java.util.Map;
import static org.apache.kafka.connect.transforms.util.Requirements.requireMap;
import static org.apache.kafka.connect.transforms.util.Requirements.requireSinkRecord;
import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct;
public abstract class InsertField<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC =
"Insert field(s) using attributes from the record metadata or a configured static value."
+ "<p/>Use the concrete transformation type designed for the record key (<code>" + Key.class.getName() + "</code>) "
+ "or value (<code>" + Value.class.getName() + "</code>).";
private interface ConfigName {
String TOPIC_FIELD = "topic.field";
String PARTITION_FIELD = "partition.field";
String OFFSET_FIELD = "offset.field";
String TIMESTAMP_FIELD = "timestamp.field";
String STATIC_FIELD = "static.field";
String STATIC_VALUE = "static.value";
}
private static final String OPTIONALITY_DOC = "Suffix with <code>!</code> to make this a required field, or <code>?</code> to keep it optional (the default).";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(ConfigName.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka topic. " + OPTIONALITY_DOC)
.define(ConfigName.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka partition. " + OPTIONALITY_DOC)
.define(ConfigName.OFFSET_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka offset - only applicable to sink connectors.<br/>" + OPTIONALITY_DOC)
.define(ConfigName.TIMESTAMP_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for record timestamp. " + OPTIONALITY_DOC)
.define(ConfigName.STATIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for static data field. " + OPTIONALITY_DOC)
.define(ConfigName.STATIC_VALUE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Static field value, if field name configured.");
private static final String PURPOSE = "field insertion";
private static final Schema OPTIONAL_TIMESTAMP_SCHEMA = Timestamp.builder().optional().build();
private static final class InsertionSpec {
final String name;
final boolean optional;
private InsertionSpec(String name, boolean optional) {
this.name = name;
this.optional = optional;
}
public static InsertionSpec parse(String spec) {
if (spec == null) return null;
if (spec.endsWith("?")) {
return new InsertionSpec(spec.substring(0, spec.length() - 1), true);
}
if (spec.endsWith("!")) {
return new InsertionSpec(spec.substring(0, spec.length() - 1), false);
}
return new InsertionSpec(spec, true);
}
}
private InsertionSpec topicField;
private InsertionSpec partitionField;
private InsertionSpec offsetField;
private InsertionSpec timestampField;
private InsertionSpec staticField;
private String staticValue;
private Cache<Schema, Schema> schemaUpdateCache;
@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
topicField = InsertionSpec.parse(config.getString(ConfigName.TOPIC_FIELD));
partitionField = InsertionSpec.parse(config.getString(ConfigName.PARTITION_FIELD));
offsetField = InsertionSpec.parse(config.getString(ConfigName.OFFSET_FIELD));
timestampField = InsertionSpec.parse(config.getString(ConfigName.TIMESTAMP_FIELD));
staticField = InsertionSpec.parse(config.getString(ConfigName.STATIC_FIELD));
staticValue = config.getString(ConfigName.STATIC_VALUE);
if (topicField == null && partitionField == null && offsetField == null && timestampField == null && staticField == null) {
throw new ConfigException("No field insertion configured");
}
if (staticField != null && staticValue == null) {
throw new ConfigException(ConfigName.STATIC_VALUE, null, "No value specified for static field: " + staticField);
}
schemaUpdateCache = new SynchronizedCache<>(new LRUCache<>(16));
}
@Override
public R apply(R record) {
if (operatingValue(record) == null) {
return record;
} else if (operatingSchema(record) == null) {
return applySchemaless(record);
} else {
return applyWithSchema(record);
}
}
private R applySchemaless(R record) {
final Map<String, Object> value = requireMap(operatingValue(record), PURPOSE);
final Map<String, Object> updatedValue = new HashMap<>(value);
if (topicField != null) {
updatedValue.put(topicField.name, record.topic());
}
if (partitionField != null && record.kafkaPartition() != null) {
updatedValue.put(partitionField.name, record.kafkaPartition());
}
if (offsetField != null) {
updatedValue.put(offsetField.name, requireSinkRecord(record, PURPOSE).kafkaOffset());
}
if (timestampField != null && record.timestamp() != null) {
updatedValue.put(timestampField.name, record.timestamp());
}
if (staticField != null && staticValue != null) {
updatedValue.put(staticField.name, staticValue);
}
return newRecord(record, null, updatedValue);
}
private R applyWithSchema(R record) {
final Struct value = requireStruct(operatingValue(record), PURPOSE);
Schema updatedSchema = schemaUpdateCache.get(value.schema());
if (updatedSchema == null) {
updatedSchema = makeUpdatedSchema(value.schema());
schemaUpdateCache.put(value.schema(), updatedSchema);
}
final Struct updatedValue = new Struct(updatedSchema);
for (Field field : value.schema().fields()) {
updatedValue.put(field.name(), value.get(field));
}
if (topicField != null) {
updatedValue.put(topicField.name, record.topic());
}
if (partitionField != null && record.kafkaPartition() != null) {
updatedValue.put(partitionField.name, record.kafkaPartition());
}
if (offsetField != null) {
updatedValue.put(offsetField.name, requireSinkRecord(record, PURPOSE).kafkaOffset());
}
if (timestampField != null && record.timestamp() != null) {
updatedValue.put(timestampField.name, new Date(record.timestamp()));
}
if (staticField != null && staticValue != null) {
updatedValue.put(staticField.name, staticValue);
}
return newRecord(record, updatedSchema, updatedValue);
}
private Schema makeUpdatedSchema(Schema schema) {
final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct());
for (Field field : schema.fields()) {
builder.field(field.name(), field.schema());
}
if (topicField != null) {
builder.field(topicField.name, topicField.optional ? Schema.OPTIONAL_STRING_SCHEMA : Schema.STRING_SCHEMA);
}
if (partitionField != null) {
builder.field(partitionField.name, partitionField.optional ? Schema.OPTIONAL_INT32_SCHEMA : Schema.INT32_SCHEMA);
}
if (offsetField != null) {
builder.field(offsetField.name, offsetField.optional ? Schema.OPTIONAL_INT64_SCHEMA : Schema.INT64_SCHEMA);
}
if (timestampField != null) {
builder.field(timestampField.name, timestampField.optional ? OPTIONAL_TIMESTAMP_SCHEMA : Timestamp.SCHEMA);
}
if (staticField != null) {
builder.field(staticField.name, staticField.optional ? Schema.OPTIONAL_STRING_SCHEMA : Schema.STRING_SCHEMA);
}
return builder.build();
}
@Override
public void close() {
schemaUpdateCache = null;
}
@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);
public static class Key<R extends ConnectRecord<R>> extends InsertField<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 InsertField<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,171 @@
/*
* 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.ConfigDef;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.transforms.util.NonEmptyListValidator;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
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 MaskField<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC =
"Mask specified fields with a valid null value for the field type (i.e. 0, false, empty string, and so on)."
+ "<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 FIELDS_CONFIG = "fields";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Names of fields to mask.");
private static final String PURPOSE = "mask fields";
private static final Map<Class<?>, Object> PRIMITIVE_VALUE_MAPPING = new HashMap<>();
static {
PRIMITIVE_VALUE_MAPPING.put(Boolean.class, Boolean.FALSE);
PRIMITIVE_VALUE_MAPPING.put(Byte.class, (byte) 0);
PRIMITIVE_VALUE_MAPPING.put(Short.class, (short) 0);
PRIMITIVE_VALUE_MAPPING.put(Integer.class, 0);
PRIMITIVE_VALUE_MAPPING.put(Long.class, 0L);
PRIMITIVE_VALUE_MAPPING.put(Float.class, 0f);
PRIMITIVE_VALUE_MAPPING.put(Double.class, 0d);
PRIMITIVE_VALUE_MAPPING.put(BigInteger.class, BigInteger.ZERO);
PRIMITIVE_VALUE_MAPPING.put(BigDecimal.class, BigDecimal.ZERO);
PRIMITIVE_VALUE_MAPPING.put(Date.class, new Date(0));
PRIMITIVE_VALUE_MAPPING.put(String.class, "");
}
private Set<String> maskedFields;
@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
maskedFields = new HashSet<>(config.getList(FIELDS_CONFIG));
}
@Override
public R apply(R record) {
if (operatingSchema(record) == null) {
return applySchemaless(record);
} else {
return applyWithSchema(record);
}
}
private R applySchemaless(R record) {
final Map<String, Object> value = requireMap(operatingValue(record), PURPOSE);
final HashMap<String, Object> updatedValue = new HashMap<>(value);
for (String field : maskedFields) {
updatedValue.put(field, masked(value.get(field)));
}
return newRecord(record, updatedValue);
}
private R applyWithSchema(R record) {
final Struct value = requireStruct(operatingValue(record), PURPOSE);
final Struct updatedValue = new Struct(value.schema());
for (Field field : value.schema().fields()) {
final Object origFieldValue = value.get(field);
updatedValue.put(field, maskedFields.contains(field.name()) ? masked(origFieldValue) : origFieldValue);
}
return newRecord(record, updatedValue);
}
private static Object masked(Object value) {
if (value == null)
return null;
Object maskedValue = PRIMITIVE_VALUE_MAPPING.get(value.getClass());
if (maskedValue == null) {
if (value instanceof List)
maskedValue = Collections.emptyList();
else if (value instanceof Map)
maskedValue = Collections.emptyMap();
else
throw new DataException("Cannot mask value of type: " + value.getClass());
}
return maskedValue;
}
@Override
public ConfigDef config() {
return CONFIG_DEF;
}
@Override
public void close() {
}
protected abstract Schema operatingSchema(R record);
protected abstract Object operatingValue(R record);
protected abstract R newRecord(R base, Object value);
public static final class Key<R extends ConnectRecord<R>> extends MaskField<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, Object updatedValue) {
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), updatedValue, record.valueSchema(), record.value(), record.timestamp());
}
}
public static final class Value<R extends ConnectRecord<R>> extends MaskField<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, Object updatedValue) {
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), record.valueSchema(), updatedValue, record.timestamp());
}
}
}

View File

@@ -0,0 +1,74 @@
/*
* 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.ConfigDef;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.transforms.util.RegexValidator;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
public class RegexRouter<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC = "Update the record topic using the configured regular expression and replacement string."
+ "<p/>Under the hood, the regex is compiled to a <code>java.util.regex.Pattern</code>. "
+ "If the pattern matches the input topic, <code>java.util.regex.Matcher#replaceFirst()</code> is used with the replacement string to obtain the new topic.";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(ConfigName.REGEX, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, new RegexValidator(), ConfigDef.Importance.HIGH,
"Regular expression to use for matching.")
.define(ConfigName.REPLACEMENT, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.HIGH,
"Replacement string.");
private interface ConfigName {
String REGEX = "regex";
String REPLACEMENT = "replacement";
}
private Pattern regex;
private String replacement;
@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
regex = Pattern.compile(config.getString(ConfigName.REGEX));
replacement = config.getString(ConfigName.REPLACEMENT);
}
@Override
public R apply(R record) {
final Matcher matcher = regex.matcher(record.topic());
if (matcher.matches()) {
final String topic = matcher.replaceFirst(replacement);
return record.newRecord(topic, record.kafkaPartition(), record.keySchema(), record.key(), record.valueSchema(), record.value(), record.timestamp());
}
return record;
}
@Override
public void close() {
}
@Override
public ConfigDef config() {
return CONFIG_DEF;
}
}

View File

@@ -0,0 +1,234 @@
/*
* 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.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.transforms.util.SchemaUtil;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
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 ReplaceField<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC = "Filter or rename fields."
+ "<p/>Use the concrete transformation type designed for the record key (<code>" + Key.class.getName() + "</code>) "
+ "or value (<code>" + Value.class.getName() + "</code>).";
interface ConfigName {
String BLACKLIST = "blacklist";
String WHITELIST = "whitelist";
String RENAME = "renames";
}
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(ConfigName.BLACKLIST, ConfigDef.Type.LIST, Collections.emptyList(), ConfigDef.Importance.MEDIUM,
"Fields to exclude. This takes precedence over the whitelist.")
.define(ConfigName.WHITELIST, ConfigDef.Type.LIST, Collections.emptyList(), ConfigDef.Importance.MEDIUM,
"Fields to include. If specified, only these fields will be used.")
.define(ConfigName.RENAME, ConfigDef.Type.LIST, Collections.emptyList(), new ConfigDef.Validator() {
@SuppressWarnings("unchecked")
@Override
public void ensureValid(String name, Object value) {
parseRenameMappings((List<String>) value);
}
@Override
public String toString() {
return "list of colon-delimited pairs, e.g. <code>foo:bar,abc:xyz</code>";
}
}, ConfigDef.Importance.MEDIUM, "Field rename mappings.");
private static final String PURPOSE = "field replacement";
private List<String> blacklist;
private List<String> whitelist;
private Map<String, String> renames;
private Map<String, String> reverseRenames;
private Cache<Schema, Schema> schemaUpdateCache;
@Override
public void configure(Map<String, ?> configs) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs);
blacklist = config.getList(ConfigName.BLACKLIST);
whitelist = config.getList(ConfigName.WHITELIST);
renames = parseRenameMappings(config.getList(ConfigName.RENAME));
reverseRenames = invert(renames);
schemaUpdateCache = new SynchronizedCache<>(new LRUCache<>(16));
}
static Map<String, String> parseRenameMappings(List<String> mappings) {
final Map<String, String> m = new HashMap<>();
for (String mapping : mappings) {
final String[] parts = mapping.split(":");
if (parts.length != 2) {
throw new ConfigException(ConfigName.RENAME, mappings, "Invalid rename mapping: " + mapping);
}
m.put(parts[0], parts[1]);
}
return m;
}
static Map<String, String> invert(Map<String, String> source) {
final Map<String, String> m = new HashMap<>();
for (Map.Entry<String, String> e : source.entrySet()) {
m.put(e.getValue(), e.getKey());
}
return m;
}
boolean filter(String fieldName) {
return !blacklist.contains(fieldName) && (whitelist.isEmpty() || whitelist.contains(fieldName));
}
String renamed(String fieldName) {
final String mapping = renames.get(fieldName);
return mapping == null ? fieldName : mapping;
}
String reverseRenamed(String fieldName) {
final String mapping = reverseRenames.get(fieldName);
return mapping == null ? fieldName : mapping;
}
@Override
public R apply(R record) {
if (operatingValue(record) == null) {
return record;
} else if (operatingSchema(record) == null) {
return applySchemaless(record);
} else {
return applyWithSchema(record);
}
}
private R applySchemaless(R record) {
final Map<String, Object> value = requireMap(operatingValue(record), PURPOSE);
final Map<String, Object> updatedValue = new HashMap<>(value.size());
for (Map.Entry<String, Object> e : value.entrySet()) {
final String fieldName = e.getKey();
if (filter(fieldName)) {
final Object fieldValue = e.getValue();
updatedValue.put(renamed(fieldName), fieldValue);
}
}
return newRecord(record, null, updatedValue);
}
private R applyWithSchema(R record) {
final Struct value = requireStruct(operatingValue(record), PURPOSE);
Schema updatedSchema = schemaUpdateCache.get(value.schema());
if (updatedSchema == null) {
updatedSchema = makeUpdatedSchema(value.schema());
schemaUpdateCache.put(value.schema(), updatedSchema);
}
final Struct updatedValue = new Struct(updatedSchema);
for (Field field : updatedSchema.fields()) {
final Object fieldValue = value.get(reverseRenamed(field.name()));
updatedValue.put(field.name(), fieldValue);
}
return newRecord(record, updatedSchema, updatedValue);
}
private Schema makeUpdatedSchema(Schema schema) {
final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct());
for (Field field : schema.fields()) {
if (filter(field.name())) {
builder.field(renamed(field.name()), field.schema());
}
}
return builder.build();
}
@Override
public ConfigDef config() {
return CONFIG_DEF;
}
@Override
public void close() {
schemaUpdateCache = null;
}
protected abstract Schema operatingSchema(R record);
protected abstract Object operatingValue(R record);
protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue);
public static class Key<R extends ConnectRecord<R>> extends ReplaceField<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 ReplaceField<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,157 @@
/*
* 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.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.Struct;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import static org.apache.kafka.connect.transforms.util.Requirements.requireSchema;
public abstract class SetSchemaMetadata<R extends ConnectRecord<R>> implements Transformation<R> {
private static final Logger log = LoggerFactory.getLogger(SetSchemaMetadata.class);
public static final String OVERVIEW_DOC =
"Set the schema name, version or both on the record's key (<code>" + Key.class.getName() + "</code>)"
+ " or value (<code>" + Value.class.getName() + "</code>) schema.";
private interface ConfigName {
String SCHEMA_NAME = "schema.name";
String SCHEMA_VERSION = "schema.version";
}
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(ConfigName.SCHEMA_NAME, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "Schema name to set.")
.define(ConfigName.SCHEMA_VERSION, ConfigDef.Type.INT, null, ConfigDef.Importance.HIGH, "Schema version to set.");
private String schemaName;
private Integer schemaVersion;
@Override
public void configure(Map<String, ?> configs) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs);
schemaName = config.getString(ConfigName.SCHEMA_NAME);
schemaVersion = config.getInt(ConfigName.SCHEMA_VERSION);
if (schemaName == null && schemaVersion == null) {
throw new ConfigException("Neither schema name nor version configured");
}
}
@Override
public R apply(R record) {
final Schema schema = operatingSchema(record);
requireSchema(schema, "updating schema metadata");
final boolean isArray = schema.type() == Schema.Type.ARRAY;
final boolean isMap = schema.type() == Schema.Type.MAP;
final Schema updatedSchema = new ConnectSchema(
schema.type(),
schema.isOptional(),
schema.defaultValue(),
schemaName != null ? schemaName : schema.name(),
schemaVersion != null ? schemaVersion : schema.version(),
schema.doc(),
schema.parameters(),
schema.fields(),
isMap ? schema.keySchema() : null,
isMap || isArray ? schema.valueSchema() : null
);
log.trace("Applying SetSchemaMetadata SMT. Original schema: {}, updated schema: {}",
schema, updatedSchema);
return newRecord(record, updatedSchema);
}
@Override
public ConfigDef config() {
return CONFIG_DEF;
}
@Override
public void close() {
}
protected abstract Schema operatingSchema(R record);
protected abstract R newRecord(R record, Schema updatedSchema);
/**
* Set the schema name, version or both on the record's key schema.
*/
public static class Key<R extends ConnectRecord<R>> extends SetSchemaMetadata<R> {
@Override
protected Schema operatingSchema(R record) {
return record.keySchema();
}
@Override
protected R newRecord(R record, Schema updatedSchema) {
Object updatedKey = updateSchemaIn(record.key(), updatedSchema);
return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedKey, record.valueSchema(), record.value(), record.timestamp());
}
}
/**
* Set the schema name, version or both on the record's value schema.
*/
public static class Value<R extends ConnectRecord<R>> extends SetSchemaMetadata<R> {
@Override
protected Schema operatingSchema(R record) {
return record.valueSchema();
}
@Override
protected R newRecord(R record, Schema updatedSchema) {
Object updatedValue = updateSchemaIn(record.value(), updatedSchema);
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp());
}
}
/**
* Utility to check the supplied key or value for references to the old Schema,
* and if so to return an updated key or value object that references the new Schema.
* Note that this method assumes that the new Schema may have a different name and/or version,
* but has fields that exactly match those of the old Schema.
* <p>
* Currently only {@link Struct} objects have references to the {@link Schema}.
*
* @param keyOrValue the key or value object; may be null
* @param updatedSchema the updated schema that has been potentially renamed
* @return the original key or value object if it does not reference the old schema, or
* a copy of the key or value object with updated references to the new schema.
*/
protected static Object updateSchemaIn(Object keyOrValue, Schema updatedSchema) {
if (keyOrValue instanceof Struct) {
Struct origStruct = (Struct) keyOrValue;
Struct newStruct = new Struct(updatedSchema);
for (Field field : updatedSchema.fields()) {
// assume both schemas have exact same fields with same names and schemas ...
newStruct.put(field, origStruct.get(field));
}
return newStruct;
}
return keyOrValue;
}
}

View File

@@ -0,0 +1,462 @@
/*
* 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.common.utils.Utils;
import org.apache.kafka.connect.connector.ConnectRecord;
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.data.Time;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.errors.ConnectException;
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.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Calendar;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.TimeZone;
import static org.apache.kafka.connect.transforms.util.Requirements.requireMap;
import static org.apache.kafka.connect.transforms.util.Requirements.requireStructOrNull;
public abstract class TimestampConverter<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC =
"Convert timestamps between different formats such as Unix epoch, strings, and Connect Date/Timestamp types."
+ "Applies to individual fields or to the entire value."
+ "<p/>Use the concrete transformation type designed for the record key (<code>" + TimestampConverter.Key.class.getName() + "</code>) "
+ "or value (<code>" + TimestampConverter.Value.class.getName() + "</code>).";
public static final String FIELD_CONFIG = "field";
private static final String FIELD_DEFAULT = "";
public static final String TARGET_TYPE_CONFIG = "target.type";
public static final String FORMAT_CONFIG = "format";
private static final String FORMAT_DEFAULT = "";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(FIELD_CONFIG, ConfigDef.Type.STRING, FIELD_DEFAULT, ConfigDef.Importance.HIGH,
"The field containing the timestamp, or empty if the entire value is a timestamp")
.define(TARGET_TYPE_CONFIG, ConfigDef.Type.STRING, ConfigDef.Importance.HIGH,
"The desired timestamp representation: string, unix, Date, Time, or Timestamp")
.define(FORMAT_CONFIG, ConfigDef.Type.STRING, FORMAT_DEFAULT, ConfigDef.Importance.MEDIUM,
"A SimpleDateFormat-compatible format for the timestamp. Used to generate the output when type=string "
+ "or used to parse the input if the input is a string.");
private static final String PURPOSE = "converting timestamp formats";
private static final String TYPE_STRING = "string";
private static final String TYPE_UNIX = "unix";
private static final String TYPE_DATE = "Date";
private static final String TYPE_TIME = "Time";
private static final String TYPE_TIMESTAMP = "Timestamp";
private static final Set<String> VALID_TYPES = new HashSet<>(Arrays.asList(TYPE_STRING, TYPE_UNIX, TYPE_DATE, TYPE_TIME, TYPE_TIMESTAMP));
private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
public static final Schema OPTIONAL_DATE_SCHEMA = org.apache.kafka.connect.data.Date.builder().optional().schema();
public static final Schema OPTIONAL_TIMESTAMP_SCHEMA = Timestamp.builder().optional().schema();
public static final Schema OPTIONAL_TIME_SCHEMA = Time.builder().optional().schema();
private interface TimestampTranslator {
/**
* Convert from the type-specific format to the universal java.util.Date format
*/
Date toRaw(Config config, Object orig);
/**
* Get the schema for this format.
*/
Schema typeSchema(boolean isOptional);
/**
* Convert from the universal java.util.Date format to the type-specific format
*/
Object toType(Config config, Date orig);
}
private static final Map<String, TimestampTranslator> TRANSLATORS = new HashMap<>();
static {
TRANSLATORS.put(TYPE_STRING, new TimestampTranslator() {
@Override
public Date toRaw(Config config, Object orig) {
if (!(orig instanceof String))
throw new DataException("Expected string timestamp to be a String, but found " + orig.getClass());
try {
return config.format.parse((String) orig);
} catch (ParseException e) {
throw new DataException("Could not parse timestamp: value (" + orig + ") does not match pattern ("
+ config.format.toPattern() + ")", e);
}
}
@Override
public Schema typeSchema(boolean isOptional) {
return isOptional ? Schema.OPTIONAL_STRING_SCHEMA : Schema.STRING_SCHEMA;
}
@Override
public String toType(Config config, Date orig) {
synchronized (config.format) {
return config.format.format(orig);
}
}
});
TRANSLATORS.put(TYPE_UNIX, new TimestampTranslator() {
@Override
public Date toRaw(Config config, Object orig) {
if (!(orig instanceof Long))
throw new DataException("Expected Unix timestamp to be a Long, but found " + orig.getClass());
return Timestamp.toLogical(Timestamp.SCHEMA, (Long) orig);
}
@Override
public Schema typeSchema(boolean isOptional) {
return isOptional ? Schema.OPTIONAL_INT64_SCHEMA : Schema.INT64_SCHEMA;
}
@Override
public Long toType(Config config, Date orig) {
return Timestamp.fromLogical(Timestamp.SCHEMA, orig);
}
});
TRANSLATORS.put(TYPE_DATE, new TimestampTranslator() {
@Override
public Date toRaw(Config config, Object orig) {
if (!(orig instanceof Date))
throw new DataException("Expected Date to be a java.util.Date, but found " + orig.getClass());
// Already represented as a java.util.Date and Connect Dates are a subset of valid java.util.Date values
return (Date) orig;
}
@Override
public Schema typeSchema(boolean isOptional) {
return isOptional ? OPTIONAL_DATE_SCHEMA : org.apache.kafka.connect.data.Date.SCHEMA;
}
@Override
public Date toType(Config config, Date orig) {
Calendar result = Calendar.getInstance(UTC);
result.setTime(orig);
result.set(Calendar.HOUR_OF_DAY, 0);
result.set(Calendar.MINUTE, 0);
result.set(Calendar.SECOND, 0);
result.set(Calendar.MILLISECOND, 0);
return result.getTime();
}
});
TRANSLATORS.put(TYPE_TIME, new TimestampTranslator() {
@Override
public Date toRaw(Config config, Object orig) {
if (!(orig instanceof Date))
throw new DataException("Expected Time to be a java.util.Date, but found " + orig.getClass());
// Already represented as a java.util.Date and Connect Times are a subset of valid java.util.Date values
return (Date) orig;
}
@Override
public Schema typeSchema(boolean isOptional) {
return isOptional ? OPTIONAL_TIME_SCHEMA : Time.SCHEMA;
}
@Override
public Date toType(Config config, Date orig) {
Calendar origCalendar = Calendar.getInstance(UTC);
origCalendar.setTime(orig);
Calendar result = Calendar.getInstance(UTC);
result.setTimeInMillis(0L);
result.set(Calendar.HOUR_OF_DAY, origCalendar.get(Calendar.HOUR_OF_DAY));
result.set(Calendar.MINUTE, origCalendar.get(Calendar.MINUTE));
result.set(Calendar.SECOND, origCalendar.get(Calendar.SECOND));
result.set(Calendar.MILLISECOND, origCalendar.get(Calendar.MILLISECOND));
return result.getTime();
}
});
TRANSLATORS.put(TYPE_TIMESTAMP, new TimestampTranslator() {
@Override
public Date toRaw(Config config, Object orig) {
if (!(orig instanceof Date))
throw new DataException("Expected Timestamp to be a java.util.Date, but found " + orig.getClass());
return (Date) orig;
}
@Override
public Schema typeSchema(boolean isOptional) {
return isOptional ? OPTIONAL_TIMESTAMP_SCHEMA : Timestamp.SCHEMA;
}
@Override
public Date toType(Config config, Date orig) {
return orig;
}
});
}
// This is a bit unusual, but allows the transformation config to be passed to static anonymous classes to customize
// their behavior
private static class Config {
Config(String field, String type, SimpleDateFormat format) {
this.field = field;
this.type = type;
this.format = format;
}
String field;
String type;
SimpleDateFormat format;
}
private Config config;
private Cache<Schema, Schema> schemaUpdateCache;
@Override
public void configure(Map<String, ?> configs) {
final SimpleConfig simpleConfig = new SimpleConfig(CONFIG_DEF, configs);
final String field = simpleConfig.getString(FIELD_CONFIG);
final String type = simpleConfig.getString(TARGET_TYPE_CONFIG);
String formatPattern = simpleConfig.getString(FORMAT_CONFIG);
schemaUpdateCache = new SynchronizedCache<>(new LRUCache<>(16));
if (!VALID_TYPES.contains(type)) {
throw new ConfigException("Unknown timestamp type in TimestampConverter: " + type + ". Valid values are "
+ Utils.join(VALID_TYPES, ", ") + ".");
}
if (type.equals(TYPE_STRING) && formatPattern.trim().isEmpty()) {
throw new ConfigException("TimestampConverter requires format option to be specified when using string timestamps");
}
SimpleDateFormat format = null;
if (formatPattern != null && !formatPattern.trim().isEmpty()) {
try {
format = new SimpleDateFormat(formatPattern);
format.setTimeZone(UTC);
} catch (IllegalArgumentException e) {
throw new ConfigException("TimestampConverter requires a SimpleDateFormat-compatible pattern for string timestamps: "
+ formatPattern, e);
}
}
config = new Config(field, type, format);
}
@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() {
}
public static class Key<R extends ConnectRecord<R>> extends TimestampConverter<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 TimestampConverter<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());
}
}
protected abstract Schema operatingSchema(R record);
protected abstract Object operatingValue(R record);
protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue);
private R applyWithSchema(R record) {
final Schema schema = operatingSchema(record);
if (config.field.isEmpty()) {
Object value = operatingValue(record);
// New schema is determined by the requested target timestamp type
Schema updatedSchema = TRANSLATORS.get(config.type).typeSchema(schema.isOptional());
return newRecord(record, updatedSchema, convertTimestamp(value, timestampTypeFromSchema(schema)));
} else {
final Struct value = requireStructOrNull(operatingValue(record), PURPOSE);
Schema updatedSchema = schemaUpdateCache.get(schema);
if (updatedSchema == null) {
SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct());
for (Field field : schema.fields()) {
if (field.name().equals(config.field)) {
builder.field(field.name(), TRANSLATORS.get(config.type).typeSchema(field.schema().isOptional()));
} else {
builder.field(field.name(), field.schema());
}
}
if (schema.isOptional())
builder.optional();
if (schema.defaultValue() != null) {
Struct updatedDefaultValue = applyValueWithSchema((Struct) schema.defaultValue(), builder);
builder.defaultValue(updatedDefaultValue);
}
updatedSchema = builder.build();
schemaUpdateCache.put(schema, updatedSchema);
}
Struct updatedValue = applyValueWithSchema(value, updatedSchema);
return newRecord(record, updatedSchema, updatedValue);
}
}
private Struct applyValueWithSchema(Struct value, Schema updatedSchema) {
if (value == null) {
return null;
}
Struct updatedValue = new Struct(updatedSchema);
for (Field field : value.schema().fields()) {
final Object updatedFieldValue;
if (field.name().equals(config.field)) {
updatedFieldValue = convertTimestamp(value.get(field), timestampTypeFromSchema(field.schema()));
} else {
updatedFieldValue = value.get(field);
}
updatedValue.put(field.name(), updatedFieldValue);
}
return updatedValue;
}
private R applySchemaless(R record) {
Object rawValue = operatingValue(record);
if (rawValue == null || config.field.isEmpty()) {
return newRecord(record, null, convertTimestamp(rawValue));
} else {
final Map<String, Object> value = requireMap(rawValue, PURPOSE);
final HashMap<String, Object> updatedValue = new HashMap<>(value);
updatedValue.put(config.field, convertTimestamp(value.get(config.field)));
return newRecord(record, null, updatedValue);
}
}
/**
* Determine the type/format of the timestamp based on the schema
*/
private String timestampTypeFromSchema(Schema schema) {
if (Timestamp.LOGICAL_NAME.equals(schema.name())) {
return TYPE_TIMESTAMP;
} else if (org.apache.kafka.connect.data.Date.LOGICAL_NAME.equals(schema.name())) {
return TYPE_DATE;
} else if (Time.LOGICAL_NAME.equals(schema.name())) {
return TYPE_TIME;
} else if (schema.type().equals(Schema.Type.STRING)) {
// If not otherwise specified, string == user-specified string format for timestamps
return TYPE_STRING;
} else if (schema.type().equals(Schema.Type.INT64)) {
// If not otherwise specified, long == unix time
return TYPE_UNIX;
}
throw new ConnectException("Schema " + schema + " does not correspond to a known timestamp type format");
}
/**
* Infer the type/format of the timestamp based on the raw Java type
*/
private String inferTimestampType(Object timestamp) {
// Note that we can't infer all types, e.g. Date/Time/Timestamp all have the same runtime representation as a
// java.util.Date
if (timestamp instanceof Date) {
return TYPE_TIMESTAMP;
} else if (timestamp instanceof Long) {
return TYPE_UNIX;
} else if (timestamp instanceof String) {
return TYPE_STRING;
}
throw new DataException("TimestampConverter does not support " + timestamp.getClass() + " objects as timestamps");
}
/**
* Convert the given timestamp to the target timestamp format.
* @param timestamp the input timestamp, may be null
* @param timestampFormat the format of the timestamp, or null if the format should be inferred
* @return the converted timestamp
*/
private Object convertTimestamp(Object timestamp, String timestampFormat) {
if (timestamp == null) {
return null;
}
if (timestampFormat == null) {
timestampFormat = inferTimestampType(timestamp);
}
TimestampTranslator sourceTranslator = TRANSLATORS.get(timestampFormat);
if (sourceTranslator == null) {
throw new ConnectException("Unsupported timestamp type: " + timestampFormat);
}
Date rawTimestamp = sourceTranslator.toRaw(config, timestamp);
TimestampTranslator targetTranslator = TRANSLATORS.get(config.type);
if (targetTranslator == null) {
throw new ConnectException("Unsupported timestamp type: " + config.type);
}
return targetTranslator.toType(config, rawTimestamp);
}
private Object convertTimestamp(Object timestamp) {
return convertTimestamp(timestamp, null);
}
}

View File

@@ -0,0 +1,102 @@
/*
* 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.ConfigDef;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.Map;
import java.util.TimeZone;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
public class TimestampRouter<R extends ConnectRecord<R>> implements Transformation<R>, AutoCloseable {
private static final Pattern TOPIC = Pattern.compile("${topic}", Pattern.LITERAL);
private static final Pattern TIMESTAMP = Pattern.compile("${timestamp}", Pattern.LITERAL);
public static final String OVERVIEW_DOC =
"Update the record's topic field as a function of the original topic value and the record timestamp."
+ "<p/>"
+ "This is mainly useful for sink connectors, since the topic field is often used to determine the equivalent entity name in the destination system"
+ "(e.g. database table or search index name).";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(ConfigName.TOPIC_FORMAT, ConfigDef.Type.STRING, "${topic}-${timestamp}", ConfigDef.Importance.HIGH,
"Format string which can contain <code>${topic}</code> and <code>${timestamp}</code> as placeholders for the topic and timestamp, respectively.")
.define(ConfigName.TIMESTAMP_FORMAT, ConfigDef.Type.STRING, "yyyyMMdd", ConfigDef.Importance.HIGH,
"Format string for the timestamp that is compatible with <code>java.text.SimpleDateFormat</code>.");
private interface ConfigName {
String TOPIC_FORMAT = "topic.format";
String TIMESTAMP_FORMAT = "timestamp.format";
}
private String topicFormat;
private ThreadLocal<SimpleDateFormat> timestampFormat;
@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
topicFormat = config.getString(ConfigName.TOPIC_FORMAT);
final String timestampFormatStr = config.getString(ConfigName.TIMESTAMP_FORMAT);
timestampFormat = new ThreadLocal<SimpleDateFormat>() {
@Override
protected SimpleDateFormat initialValue() {
final SimpleDateFormat fmt = new SimpleDateFormat(timestampFormatStr);
fmt.setTimeZone(TimeZone.getTimeZone("UTC"));
return fmt;
}
};
}
@Override
public R apply(R record) {
final Long timestamp = record.timestamp();
if (timestamp == null) {
throw new DataException("Timestamp missing on record: " + record);
}
final String formattedTimestamp = timestampFormat.get().format(new Date(timestamp));
final String replace1 = TOPIC.matcher(topicFormat).replaceAll(Matcher.quoteReplacement(record.topic()));
final String updatedTopic = TIMESTAMP.matcher(replace1).replaceAll(Matcher.quoteReplacement(formattedTimestamp));
return record.newRecord(
updatedTopic, record.kafkaPartition(),
record.keySchema(), record.key(),
record.valueSchema(), record.value(),
record.timestamp()
);
}
@Override
public void close() {
timestampFormat = null;
}
@Override
public ConfigDef config() {
return CONFIG_DEF;
}
}

View File

@@ -0,0 +1,115 @@
/*
* 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.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.NonEmptyListValidator;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.HashMap;
import java.util.List;
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 class ValueToKey<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC = "Replace the record key with a new key formed from a subset of fields in the record value.";
public static final String FIELDS_CONFIG = "fields";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH,
"Field names on the record value to extract as the record key.");
private static final String PURPOSE = "copying fields from value to key";
private List<String> fields;
private Cache<Schema, Schema> valueToKeySchemaCache;
@Override
public void configure(Map<String, ?> configs) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs);
fields = config.getList(FIELDS_CONFIG);
valueToKeySchemaCache = new SynchronizedCache<>(new LRUCache<>(16));
}
@Override
public R apply(R record) {
if (record.valueSchema() == null) {
return applySchemaless(record);
} else {
return applyWithSchema(record);
}
}
private R applySchemaless(R record) {
final Map<String, Object> value = requireMap(record.value(), PURPOSE);
final Map<String, Object> key = new HashMap<>(fields.size());
for (String field : fields) {
key.put(field, value.get(field));
}
return record.newRecord(record.topic(), record.kafkaPartition(), null, key, record.valueSchema(), record.value(), record.timestamp());
}
private R applyWithSchema(R record) {
final Struct value = requireStruct(record.value(), PURPOSE);
Schema keySchema = valueToKeySchemaCache.get(value.schema());
if (keySchema == null) {
final SchemaBuilder keySchemaBuilder = SchemaBuilder.struct();
for (String field : fields) {
final Field fieldFromValue = value.schema().field(field);
if (fieldFromValue == null) {
throw new DataException("Field does not exist: " + field);
}
keySchemaBuilder.field(field, fieldFromValue.schema());
}
keySchema = keySchemaBuilder.build();
valueToKeySchemaCache.put(value.schema(), keySchema);
}
final Struct key = new Struct(keySchema);
for (String field : fields) {
key.put(field, value.get(field));
}
return record.newRecord(record.topic(), record.kafkaPartition(), keySchema, key, value.schema(), value, record.timestamp());
}
@Override
public ConfigDef config() {
return CONFIG_DEF;
}
@Override
public void close() {
valueToKeySchemaCache = null;
}
}

View File

@@ -0,0 +1,38 @@
/*
* 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.util;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigException;
import java.util.List;
public class NonEmptyListValidator implements ConfigDef.Validator {
@Override
public void ensureValid(String name, Object value) {
if (value == null || ((List) value).isEmpty()) {
throw new ConfigException(name, value, "Empty list");
}
}
@Override
public String toString() {
return "non-empty list";
}
}

View File

@@ -0,0 +1,40 @@
/*
* 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.util;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigException;
import java.util.regex.Pattern;
public class RegexValidator implements ConfigDef.Validator {
@Override
public void ensureValid(String name, Object value) {
try {
Pattern.compile((String) value);
} catch (Exception e) {
throw new ConfigException(name, value, "Invalid regex: " + e.getMessage());
}
}
@Override
public String toString() {
return "valid regex";
}
}

View File

@@ -0,0 +1,75 @@
/*
* 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.util;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.sink.SinkRecord;
import java.util.Map;
public class Requirements {
public static void requireSchema(Schema schema, String purpose) {
if (schema == null) {
throw new DataException("Schema required for [" + purpose + "]");
}
}
@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));
}
return (Map<String, Object>) value;
}
public static Map<String, Object> requireMapOrNull(Object value, String purpose) {
if (value == null) {
return null;
}
return requireMap(value, purpose);
}
public static Struct requireStruct(Object value, String purpose) {
if (!(value instanceof Struct)) {
throw new DataException("Only Struct objects supported for [" + purpose + "], found: " + nullSafeClassName(value));
}
return (Struct) value;
}
public static Struct requireStructOrNull(Object value, String purpose) {
if (value == null) {
return null;
}
return requireStruct(value, purpose);
}
public static SinkRecord requireSinkRecord(ConnectRecord<?> record, String purpose) {
if (!(record instanceof SinkRecord)) {
throw new DataException("Only SinkRecord supported for [" + purpose + "], found: " + nullSafeClassName(record));
}
return (SinkRecord) record;
}
private static String nullSafeClassName(Object x) {
return x == null ? "null" : x.getClass().getName();
}
}

View File

@@ -0,0 +1,43 @@
/*
* 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.util;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
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());
builder.doc(source.doc());
final Map<String, String> params = source.parameters();
if (params != null) {
builder.parameters(params);
}
return builder;
}
}

View File

@@ -0,0 +1,33 @@
/*
* 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.util;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import java.util.Map;
/**
* A barebones concrete implementation of {@link AbstractConfig}.
*/
public class SimpleConfig extends AbstractConfig {
public SimpleConfig(ConfigDef configDef, Map<?, ?> originals) {
super(configDef, originals, false);
}
}

View File

@@ -0,0 +1,519 @@
/*
* 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 java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.connect.data.Decimal;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Schema.Type;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.data.Time;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.data.Values;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.After;
import org.junit.Test;
import java.math.BigDecimal;
import java.util.Collections;
import java.util.Date;
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 {
private final Cast<SourceRecord> xformKey = new Cast.Key<>();
private final Cast<SourceRecord> xformValue = new Cast.Value<>();
private static final long MILLIS_PER_HOUR = TimeUnit.HOURS.toMillis(1);
private static final long MILLIS_PER_DAY = TimeUnit.DAYS.toMillis(1);
@After
public void teardown() {
xformKey.close();
xformValue.close();
}
@Test(expected = ConfigException.class)
public void testConfigEmpty() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, ""));
}
@Test(expected = ConfigException.class)
public void testConfigInvalidSchemaType() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:faketype"));
}
@Test(expected = ConfigException.class)
public void testConfigInvalidTargetType() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:array"));
}
@Test(expected = ConfigException.class)
public void testUnsupportedTargetType() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:bytes"));
}
@Test(expected = ConfigException.class)
public void testConfigInvalidMap() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int8:extra"));
}
@Test(expected = ConfigException.class)
public void testConfigMixWholeAndFieldTransformation() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int8,int32"));
}
@Test
public void castWholeRecordKeyWithSchema() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
SourceRecord transformed = xformKey.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
SourceRecord transformed = xformValue.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int16"));
SourceRecord transformed = xformValue.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int32"));
SourceRecord transformed = xformValue.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int64"));
SourceRecord transformed = xformValue.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "float32"));
SourceRecord transformed = xformValue.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "float64"));
SourceRecord transformed = xformValue.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "boolean"));
SourceRecord transformed = xformValue.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "boolean"));
SourceRecord transformed = xformValue.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "string"));
SourceRecord transformed = xformValue.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 castWholeBigDecimalRecordValueWithSchemaString() {
BigDecimal bigDecimal = new BigDecimal(42);
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "string"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
Decimal.schema(bigDecimal.scale()), bigDecimal));
assertEquals(Schema.Type.STRING, transformed.valueSchema().type());
assertEquals("42", transformed.value());
}
@Test
public void castWholeDateRecordValueWithSchemaString() {
Date timestamp = new Date(MILLIS_PER_DAY + 1); // day + 1msec to get a timestamp formatting.
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "string"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
Timestamp.SCHEMA, timestamp));
assertEquals(Schema.Type.STRING, transformed.valueSchema().type());
assertEquals(Values.dateFormatFor(timestamp).format(timestamp), transformed.value());
}
@Test
public void castWholeRecordDefaultValue() {
// Validate default value in schema is correctly converted
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int32"));
SourceRecord transformed = xformValue.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() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
SourceRecord transformed = xformKey.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() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 42));
assertNull(transformed.valueSchema());
assertEquals((byte) 42, transformed.value());
}
@Test
public void castWholeRecordValueSchemalessInt16() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int16"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 42));
assertNull(transformed.valueSchema());
assertEquals((short) 42, transformed.value());
}
@Test
public void castWholeRecordValueSchemalessInt32() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int32"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 42));
assertNull(transformed.valueSchema());
assertEquals(42, transformed.value());
}
@Test
public void castWholeRecordValueSchemalessInt64() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int64"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 42));
assertNull(transformed.valueSchema());
assertEquals((long) 42, transformed.value());
}
@Test
public void castWholeRecordValueSchemalessFloat32() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "float32"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 42));
assertNull(transformed.valueSchema());
assertEquals(42.f, transformed.value());
}
@Test
public void castWholeRecordValueSchemalessFloat64() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "float64"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 42));
assertNull(transformed.valueSchema());
assertEquals(42., transformed.value());
}
@Test
public void castWholeRecordValueSchemalessBooleanTrue() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "boolean"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 42));
assertNull(transformed.valueSchema());
assertEquals(true, transformed.value());
}
@Test
public void castWholeRecordValueSchemalessBooleanFalse() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "boolean"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 0));
assertNull(transformed.valueSchema());
assertEquals(false, transformed.value());
}
@Test
public void castWholeRecordValueSchemalessString() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "string"));
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
null, 42));
assertNull(transformed.valueSchema());
assertEquals("42", transformed.value());
}
@Test(expected = DataException.class)
public void castWholeRecordValueSchemalessUnsupportedType() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
xformValue.apply(new SourceRecord(null, null, "topic", 0, null, Collections.singletonList("foo")));
}
@Test
public void castLogicalToPrimitive() {
List<String> specParts = Arrays.asList(
"date_to_int32:int32", // Cast to underlying representation
"timestamp_to_int64:int64", // Cast to underlying representation
"time_to_int64:int64", // Cast to wider datatype than underlying representation
"decimal_to_int32:int32", // Cast to narrower datatype with data loss
"timestamp_to_float64:float64", // loss of precision casting to double
"null_timestamp_to_int32:int32"
);
Date day = new Date(MILLIS_PER_DAY);
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG,
String.join(",", specParts)));
SchemaBuilder builder = SchemaBuilder.struct();
builder.field("date_to_int32", org.apache.kafka.connect.data.Date.SCHEMA);
builder.field("timestamp_to_int64", Timestamp.SCHEMA);
builder.field("time_to_int64", Time.SCHEMA);
builder.field("decimal_to_int32", Decimal.schema(new BigDecimal((long) Integer.MAX_VALUE + 1).scale()));
builder.field("timestamp_to_float64", Timestamp.SCHEMA);
builder.field("null_timestamp_to_int32", Timestamp.builder().optional().build());
Schema supportedTypesSchema = builder.build();
Struct recordValue = new Struct(supportedTypesSchema);
recordValue.put("date_to_int32", day);
recordValue.put("timestamp_to_int64", new Date(0));
recordValue.put("time_to_int64", new Date(1));
recordValue.put("decimal_to_int32", new BigDecimal((long) Integer.MAX_VALUE + 1));
recordValue.put("timestamp_to_float64", new Date(Long.MAX_VALUE));
recordValue.put("null_timestamp_to_int32", null);
SourceRecord transformed = xformValue.apply(
new SourceRecord(null, null, "topic", 0,
supportedTypesSchema, recordValue));
assertEquals(1, ((Struct) transformed.value()).get("date_to_int32"));
assertEquals(0L, ((Struct) transformed.value()).get("timestamp_to_int64"));
assertEquals(1L, ((Struct) transformed.value()).get("time_to_int64"));
assertEquals(Integer.MIN_VALUE, ((Struct) transformed.value()).get("decimal_to_int32"));
assertEquals(9.223372036854776E18, ((Struct) transformed.value()).get("timestamp_to_float64"));
assertNull(((Struct) transformed.value()).get("null_timestamp_to_int32"));
Schema transformedSchema = ((Struct) transformed.value()).schema();
assertEquals(Type.INT32, transformedSchema.field("date_to_int32").schema().type());
assertEquals(Type.INT64, transformedSchema.field("timestamp_to_int64").schema().type());
assertEquals(Type.INT64, transformedSchema.field("time_to_int64").schema().type());
assertEquals(Type.INT32, transformedSchema.field("decimal_to_int32").schema().type());
assertEquals(Type.FLOAT64, transformedSchema.field("timestamp_to_float64").schema().type());
assertEquals(Type.INT32, transformedSchema.field("null_timestamp_to_int32").schema().type());
}
@Test
public void castLogicalToString() {
Date date = new Date(MILLIS_PER_DAY);
Date time = new Date(MILLIS_PER_HOUR);
Date timestamp = new Date();
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG,
"date:string,decimal:string,time:string,timestamp:string"));
SchemaBuilder builder = SchemaBuilder.struct();
builder.field("date", org.apache.kafka.connect.data.Date.SCHEMA);
builder.field("decimal", Decimal.schema(new BigDecimal(1982).scale()));
builder.field("time", Time.SCHEMA);
builder.field("timestamp", Timestamp.SCHEMA);
Schema supportedTypesSchema = builder.build();
Struct recordValue = new Struct(supportedTypesSchema);
recordValue.put("date", date);
recordValue.put("decimal", new BigDecimal(1982));
recordValue.put("time", time);
recordValue.put("timestamp", timestamp);
SourceRecord transformed = xformValue.apply(
new SourceRecord(null, null, "topic", 0,
supportedTypesSchema, recordValue));
assertEquals(Values.dateFormatFor(date).format(date), ((Struct) transformed.value()).get("date"));
assertEquals("1982", ((Struct) transformed.value()).get("decimal"));
assertEquals(Values.dateFormatFor(time).format(time), ((Struct) transformed.value()).get("time"));
assertEquals(Values.dateFormatFor(timestamp).format(timestamp), ((Struct) transformed.value()).get("timestamp"));
Schema transformedSchema = ((Struct) transformed.value()).schema();
assertEquals(Type.STRING, transformedSchema.field("date").schema().type());
assertEquals(Type.STRING, transformedSchema.field("decimal").schema().type());
assertEquals(Type.STRING, transformedSchema.field("time").schema().type());
assertEquals(Type.STRING, transformedSchema.field("timestamp").schema().type());
}
@Test
public void castFieldsWithSchema() {
Date day = new Date(MILLIS_PER_DAY);
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8:int16,int16:int32,int32:int64,int64:boolean,float32:float64,float64:boolean,boolean:int8,string:int32,bigdecimal:string,date:string,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("bigdecimal", Decimal.schema(new BigDecimal(42).scale()));
builder.field("date", org.apache.kafka.connect.data.Date.SCHEMA);
builder.field("optional", Schema.OPTIONAL_FLOAT32_SCHEMA);
builder.field("timestamp", Timestamp.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("bigdecimal", new BigDecimal(42));
recordValue.put("date", day);
recordValue.put("string", "42");
recordValue.put("timestamp", new Date(0));
// optional field intentionally omitted
SourceRecord transformed = xformValue.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"));
assertEquals("42", ((Struct) transformed.value()).get("bigdecimal"));
assertEquals(Values.dateFormatFor(day).format(day), ((Struct) transformed.value()).get("date"));
assertEquals(new Date(0), ((Struct) transformed.value()).get("timestamp"));
assertNull(((Struct) transformed.value()).get("optional"));
Schema transformedSchema = ((Struct) transformed.value()).schema();
assertEquals(Schema.INT16_SCHEMA.type(), transformedSchema.field("int8").schema().type());
assertEquals(Schema.OPTIONAL_INT32_SCHEMA.type(), transformedSchema.field("int16").schema().type());
assertEquals(Schema.INT64_SCHEMA.type(), transformedSchema.field("int32").schema().type());
assertEquals(Schema.BOOLEAN_SCHEMA.type(), transformedSchema.field("int64").schema().type());
assertEquals(Schema.FLOAT64_SCHEMA.type(), transformedSchema.field("float32").schema().type());
assertEquals(Schema.BOOLEAN_SCHEMA.type(), transformedSchema.field("float64").schema().type());
assertEquals(Schema.INT8_SCHEMA.type(), transformedSchema.field("boolean").schema().type());
assertEquals(Schema.INT32_SCHEMA.type(), transformedSchema.field("string").schema().type());
assertEquals(Schema.STRING_SCHEMA.type(), transformedSchema.field("bigdecimal").schema().type());
assertEquals(Schema.STRING_SCHEMA.type(), transformedSchema.field("date").schema().type());
assertEquals(Schema.OPTIONAL_INT32_SCHEMA.type(), transformedSchema.field("optional").schema().type());
// The following fields are not changed
assertEquals(Timestamp.SCHEMA.type(), transformedSchema.field("timestamp").schema().type());
}
@SuppressWarnings("unchecked")
@Test
public void castFieldsSchemaless() {
xformValue.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 = xformValue.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,116 @@
/*
* 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.sink.SinkRecord;
import org.junit.After;
import org.junit.Test;
import java.util.Collections;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.fail;
public class ExtractFieldTest {
private final ExtractField<SinkRecord> xform = new ExtractField.Key<>();
@After
public void teardown() {
xform.close();
}
@Test
public void schemaless() {
xform.configure(Collections.singletonMap("field", "magic"));
final SinkRecord record = new SinkRecord("test", 0, null, Collections.singletonMap("magic", 42), null, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertNull(transformedRecord.keySchema());
assertEquals(42, transformedRecord.key());
}
@Test
public void testNullSchemaless() {
xform.configure(Collections.singletonMap("field", "magic"));
final Map<String, Object> key = null;
final SinkRecord record = new SinkRecord("test", 0, null, key, null, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertNull(transformedRecord.keySchema());
assertNull(transformedRecord.key());
}
@Test
public void withSchema() {
xform.configure(Collections.singletonMap("field", "magic"));
final Schema keySchema = SchemaBuilder.struct().field("magic", Schema.INT32_SCHEMA).build();
final Struct key = new Struct(keySchema).put("magic", 42);
final SinkRecord record = new SinkRecord("test", 0, keySchema, key, null, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertEquals(Schema.INT32_SCHEMA, transformedRecord.keySchema());
assertEquals(42, transformedRecord.key());
}
@Test
public void testNullWithSchema() {
xform.configure(Collections.singletonMap("field", "magic"));
final Schema keySchema = SchemaBuilder.struct().field("magic", Schema.INT32_SCHEMA).optional().build();
final Struct key = null;
final SinkRecord record = new SinkRecord("test", 0, keySchema, key, null, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertEquals(Schema.INT32_SCHEMA, transformedRecord.keySchema());
assertNull(transformedRecord.key());
}
@Test
public void nonExistentFieldSchemalessShouldReturnNull() {
xform.configure(Collections.singletonMap("field", "nonexistent"));
final SinkRecord record = new SinkRecord("test", 0, null, Collections.singletonMap("magic", 42), null, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertNull(transformedRecord.keySchema());
assertNull(transformedRecord.key());
}
@Test
public void nonExistentFieldWithSchemaShouldFail() {
xform.configure(Collections.singletonMap("field", "nonexistent"));
final Schema keySchema = SchemaBuilder.struct().field("magic", Schema.INT32_SCHEMA).build();
final Struct key = new Struct(keySchema).put("magic", 42);
final SinkRecord record = new SinkRecord("test", 0, keySchema, key, null, null, 0);
try {
xform.apply(record);
fail("Expected exception wasn't raised");
} catch (IllegalArgumentException iae) {
assertEquals("Unknown field: nonexistent", iae.getMessage());
}
}
}

View File

@@ -0,0 +1,325 @@
/*
* 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.After;
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 {
private final Flatten<SourceRecord> xformKey = new Flatten.Key<>();
private final Flatten<SourceRecord> xformValue = new Flatten.Value<>();
@After
public void teardown() {
xformKey.close();
xformValue.close();
}
@Test(expected = DataException.class)
public void topLevelStructRequired() {
xformValue.configure(Collections.<String, String>emptyMap());
xformValue.apply(new SourceRecord(null, null, "topic", 0, Schema.INT32_SCHEMA, 42));
}
@Test(expected = DataException.class)
public void topLevelMapRequired() {
xformValue.configure(Collections.<String, String>emptyMap());
xformValue.apply(new SourceRecord(null, null, "topic", 0, null, 42));
}
@Test
public void testNestedStruct() {
xformValue.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 = xformValue.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() {
xformValue.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 = xformValue.apply(new SourceRecord(null, null,
"topic", 0,
null, twoLevelNestedMap));
assertNull(transformed.valueSchema());
assertTrue(transformed.value() instanceof Map);
@SuppressWarnings("unchecked")
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() {
xformValue.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 = xformValue.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 testOptionalStruct() {
xformValue.configure(Collections.<String, String>emptyMap());
SchemaBuilder builder = SchemaBuilder.struct().optional();
builder.field("opt_int32", Schema.OPTIONAL_INT32_SCHEMA);
Schema schema = builder.build();
SourceRecord transformed = xformValue.apply(new SourceRecord(null, null,
"topic", 0,
schema, null));
assertEquals(Schema.Type.STRUCT, transformed.valueSchema().type());
assertNull(transformed.value());
}
@Test
public void testOptionalNestedStruct() {
xformValue.configure(Collections.<String, String>emptyMap());
SchemaBuilder builder = SchemaBuilder.struct().optional();
builder.field("opt_int32", Schema.OPTIONAL_INT32_SCHEMA);
Schema supportedTypesSchema = builder.build();
builder = SchemaBuilder.struct();
builder.field("B", supportedTypesSchema);
Schema oneLevelNestedSchema = builder.build();
Struct oneLevelNestedStruct = new Struct(oneLevelNestedSchema);
oneLevelNestedStruct.put("B", null);
SourceRecord transformed = xformValue.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() {
xformValue.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 = xformValue.apply(new SourceRecord(null, null,
"topic", 0,
null, oneLevelNestedMap));
assertNull(transformed.valueSchema());
assertTrue(transformed.value() instanceof Map);
@SuppressWarnings("unchecked")
Map<String, Object> transformedMap = (Map<String, Object>) transformed.value();
assertNull(transformedMap.get("B.opt_int32"));
}
@Test
public void testKey() {
xformKey.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 = xformKey.apply(src);
assertNull(transformed.keySchema());
assertTrue(transformed.key() instanceof Map);
@SuppressWarnings("unchecked")
Map<String, Object> transformedMap = (Map<String, Object>) transformed.key();
assertEquals(12, transformedMap.get("A.B"));
}
@Test(expected = DataException.class)
public void testUnsupportedTypeInMap() {
xformValue.configure(Collections.<String, String>emptyMap());
Object value = Collections.singletonMap("foo", Arrays.asList("bar", "baz"));
xformValue.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
xformValue.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 = xformValue.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());
}
@Test
public void tombstoneEventWithoutSchemaShouldPassThrough() {
xformValue.configure(Collections.<String, String>emptyMap());
final SourceRecord record = new SourceRecord(null, null, "test", 0,
null, null);
final SourceRecord transformedRecord = xformValue.apply(record);
assertEquals(null, transformedRecord.value());
assertEquals(null, transformedRecord.valueSchema());
}
@Test
public void tombstoneEventWithSchemaShouldPassThrough() {
xformValue.configure(Collections.<String, String>emptyMap());
final Schema simpleStructSchema = SchemaBuilder.struct().name("name").version(1).doc("doc").field("magic", Schema.OPTIONAL_INT64_SCHEMA).build();
final SourceRecord record = new SourceRecord(null, null, "test", 0,
simpleStructSchema, null);
final SourceRecord transformedRecord = xformValue.apply(record);
assertEquals(null, transformedRecord.value());
assertEquals(simpleStructSchema, transformedRecord.valueSchema());
}
}

View File

@@ -0,0 +1,61 @@
/*
* 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.Struct;
import org.apache.kafka.connect.sink.SinkRecord;
import org.junit.After;
import org.junit.Test;
import java.util.Collections;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
public class HoistFieldTest {
private final HoistField<SinkRecord> xform = new HoistField.Key<>();
@After
public void teardown() {
xform.close();
}
@Test
public void schemaless() {
xform.configure(Collections.singletonMap("field", "magic"));
final SinkRecord record = new SinkRecord("test", 0, null, 42, null, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertNull(transformedRecord.keySchema());
assertEquals(Collections.singletonMap("magic", 42), transformedRecord.key());
}
@Test
public void withSchema() {
xform.configure(Collections.singletonMap("field", "magic"));
final SinkRecord record = new SinkRecord("test", 0, Schema.INT32_SCHEMA, 42, null, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertEquals(Schema.Type.STRUCT, transformedRecord.keySchema().type());
assertEquals(record.keySchema(), transformedRecord.keySchema().field("magic").schema());
assertEquals(42, ((Struct) transformedRecord.key()).get("magic"));
}
}

View File

@@ -0,0 +1,200 @@
/*
* 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.data.Timestamp;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.After;
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.assertSame;
public class InsertFieldTest {
private InsertField<SourceRecord> xformKey = new InsertField.Key<>();
private InsertField<SourceRecord> xformValue = new InsertField.Value<>();
@After
public void teardown() {
xformValue.close();
}
@Test(expected = DataException.class)
public void topLevelStructRequired() {
xformValue.configure(Collections.singletonMap("topic.field", "topic_field"));
xformValue.apply(new SourceRecord(null, null, "", 0, Schema.INT32_SCHEMA, 42));
}
@Test
public void copySchemaAndInsertConfiguredFields() {
final Map<String, Object> props = new HashMap<>();
props.put("topic.field", "topic_field!");
props.put("partition.field", "partition_field");
props.put("timestamp.field", "timestamp_field?");
props.put("static.field", "instance_id");
props.put("static.value", "my-instance-id");
xformValue.configure(props);
final Schema simpleStructSchema = SchemaBuilder.struct().name("name").version(1).doc("doc").field("magic", Schema.OPTIONAL_INT64_SCHEMA).build();
final Struct simpleStruct = new Struct(simpleStructSchema).put("magic", 42L);
final SourceRecord record = new SourceRecord(null, null, "test", 0, simpleStructSchema, simpleStruct);
final SourceRecord transformedRecord = xformValue.apply(record);
assertEquals(simpleStructSchema.name(), transformedRecord.valueSchema().name());
assertEquals(simpleStructSchema.version(), transformedRecord.valueSchema().version());
assertEquals(simpleStructSchema.doc(), transformedRecord.valueSchema().doc());
assertEquals(Schema.OPTIONAL_INT64_SCHEMA, transformedRecord.valueSchema().field("magic").schema());
assertEquals(42L, ((Struct) transformedRecord.value()).getInt64("magic").longValue());
assertEquals(Schema.STRING_SCHEMA, transformedRecord.valueSchema().field("topic_field").schema());
assertEquals("test", ((Struct) transformedRecord.value()).getString("topic_field"));
assertEquals(Schema.OPTIONAL_INT32_SCHEMA, transformedRecord.valueSchema().field("partition_field").schema());
assertEquals(0, ((Struct) transformedRecord.value()).getInt32("partition_field").intValue());
assertEquals(Timestamp.builder().optional().build(), transformedRecord.valueSchema().field("timestamp_field").schema());
assertEquals(null, ((Struct) transformedRecord.value()).getInt64("timestamp_field"));
assertEquals(Schema.OPTIONAL_STRING_SCHEMA, transformedRecord.valueSchema().field("instance_id").schema());
assertEquals("my-instance-id", ((Struct) transformedRecord.value()).getString("instance_id"));
// Exercise caching
final SourceRecord transformedRecord2 = xformValue.apply(
new SourceRecord(null, null, "test", 1, simpleStructSchema, new Struct(simpleStructSchema)));
assertSame(transformedRecord.valueSchema(), transformedRecord2.valueSchema());
}
@Test
public void schemalessInsertConfiguredFields() {
final Map<String, Object> props = new HashMap<>();
props.put("topic.field", "topic_field!");
props.put("partition.field", "partition_field");
props.put("timestamp.field", "timestamp_field?");
props.put("static.field", "instance_id");
props.put("static.value", "my-instance-id");
xformValue.configure(props);
final SourceRecord record = new SourceRecord(null, null, "test", 0,
null, Collections.singletonMap("magic", 42L));
final SourceRecord transformedRecord = xformValue.apply(record);
assertEquals(42L, ((Map<?, ?>) transformedRecord.value()).get("magic"));
assertEquals("test", ((Map<?, ?>) transformedRecord.value()).get("topic_field"));
assertEquals(0, ((Map<?, ?>) transformedRecord.value()).get("partition_field"));
assertEquals(null, ((Map<?, ?>) transformedRecord.value()).get("timestamp_field"));
assertEquals("my-instance-id", ((Map<?, ?>) transformedRecord.value()).get("instance_id"));
}
@Test
public void insertConfiguredFieldsIntoTombstoneEventWithoutSchemaLeavesValueUnchanged() {
final Map<String, Object> props = new HashMap<>();
props.put("topic.field", "topic_field!");
props.put("partition.field", "partition_field");
props.put("timestamp.field", "timestamp_field?");
props.put("static.field", "instance_id");
props.put("static.value", "my-instance-id");
xformValue.configure(props);
final SourceRecord record = new SourceRecord(null, null, "test", 0,
null, null);
final SourceRecord transformedRecord = xformValue.apply(record);
assertEquals(null, transformedRecord.value());
assertEquals(null, transformedRecord.valueSchema());
}
@Test
public void insertConfiguredFieldsIntoTombstoneEventWithSchemaLeavesValueUnchanged() {
final Map<String, Object> props = new HashMap<>();
props.put("topic.field", "topic_field!");
props.put("partition.field", "partition_field");
props.put("timestamp.field", "timestamp_field?");
props.put("static.field", "instance_id");
props.put("static.value", "my-instance-id");
xformValue.configure(props);
final Schema simpleStructSchema = SchemaBuilder.struct().name("name").version(1).doc("doc").field("magic", Schema.OPTIONAL_INT64_SCHEMA).build();
final SourceRecord record = new SourceRecord(null, null, "test", 0,
simpleStructSchema, null);
final SourceRecord transformedRecord = xformValue.apply(record);
assertEquals(null, transformedRecord.value());
assertEquals(simpleStructSchema, transformedRecord.valueSchema());
}
@Test
public void insertKeyFieldsIntoTombstoneEvent() {
final Map<String, Object> props = new HashMap<>();
props.put("topic.field", "topic_field!");
props.put("partition.field", "partition_field");
props.put("timestamp.field", "timestamp_field?");
props.put("static.field", "instance_id");
props.put("static.value", "my-instance-id");
xformKey.configure(props);
final SourceRecord record = new SourceRecord(null, null, "test", 0,
null, Collections.singletonMap("magic", 42L), null, null);
final SourceRecord transformedRecord = xformKey.apply(record);
assertEquals(42L, ((Map<?, ?>) transformedRecord.key()).get("magic"));
assertEquals("test", ((Map<?, ?>) transformedRecord.key()).get("topic_field"));
assertEquals(0, ((Map<?, ?>) transformedRecord.key()).get("partition_field"));
assertEquals(null, ((Map<?, ?>) transformedRecord.key()).get("timestamp_field"));
assertEquals("my-instance-id", ((Map<?, ?>) transformedRecord.key()).get("instance_id"));
assertEquals(null, transformedRecord.value());
}
@Test
public void insertIntoNullKeyLeavesRecordUnchanged() {
final Map<String, Object> props = new HashMap<>();
props.put("topic.field", "topic_field!");
props.put("partition.field", "partition_field");
props.put("timestamp.field", "timestamp_field?");
props.put("static.field", "instance_id");
props.put("static.value", "my-instance-id");
xformKey.configure(props);
final SourceRecord record = new SourceRecord(null, null, "test", 0,
null, null, null, Collections.singletonMap("magic", 42L));
final SourceRecord transformedRecord = xformKey.apply(record);
assertSame(record, transformedRecord);
}
}

View File

@@ -0,0 +1,156 @@
/*
* 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.Decimal;
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.data.Time;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.sink.SinkRecord;
import org.junit.Test;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
public class MaskFieldTest {
private static MaskField<SinkRecord> transform(List<String> fields) {
final MaskField<SinkRecord> xform = new MaskField.Value<>();
xform.configure(Collections.singletonMap("fields", fields));
return xform;
}
private static SinkRecord record(Schema schema, Object value) {
return new SinkRecord("", 0, null, null, schema, value, 0);
}
@Test
public void schemaless() {
final Map<String, Object> value = new HashMap<>();
value.put("magic", 42);
value.put("bool", true);
value.put("byte", (byte) 42);
value.put("short", (short) 42);
value.put("int", 42);
value.put("long", 42L);
value.put("float", 42f);
value.put("double", 42d);
value.put("string", "blabla");
value.put("date", new Date());
value.put("bigint", new BigInteger("42"));
value.put("bigdec", new BigDecimal("42.0"));
value.put("list", Collections.singletonList(42));
value.put("map", Collections.singletonMap("key", "value"));
final List<String> maskFields = new ArrayList<>(value.keySet());
maskFields.remove("magic");
@SuppressWarnings("unchecked")
final Map<String, Object> updatedValue = (Map) transform(maskFields).apply(record(null, value)).value();
assertEquals(42, updatedValue.get("magic"));
assertEquals(false, updatedValue.get("bool"));
assertEquals((byte) 0, updatedValue.get("byte"));
assertEquals((short) 0, updatedValue.get("short"));
assertEquals(0, updatedValue.get("int"));
assertEquals(0L, updatedValue.get("long"));
assertEquals(0f, updatedValue.get("float"));
assertEquals(0d, updatedValue.get("double"));
assertEquals("", updatedValue.get("string"));
assertEquals(new Date(0), updatedValue.get("date"));
assertEquals(BigInteger.ZERO, updatedValue.get("bigint"));
assertEquals(BigDecimal.ZERO, updatedValue.get("bigdec"));
assertEquals(Collections.emptyList(), updatedValue.get("list"));
assertEquals(Collections.emptyMap(), updatedValue.get("map"));
}
@Test
public void withSchema() {
Schema schema = SchemaBuilder.struct()
.field("magic", Schema.INT32_SCHEMA)
.field("bool", Schema.BOOLEAN_SCHEMA)
.field("byte", Schema.INT8_SCHEMA)
.field("short", Schema.INT16_SCHEMA)
.field("int", Schema.INT32_SCHEMA)
.field("long", Schema.INT64_SCHEMA)
.field("float", Schema.FLOAT32_SCHEMA)
.field("double", Schema.FLOAT64_SCHEMA)
.field("string", Schema.STRING_SCHEMA)
.field("date", org.apache.kafka.connect.data.Date.SCHEMA)
.field("time", Time.SCHEMA)
.field("timestamp", Timestamp.SCHEMA)
.field("decimal", Decimal.schema(0))
.field("array", SchemaBuilder.array(Schema.INT32_SCHEMA))
.field("map", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA))
.build();
final Struct value = new Struct(schema);
value.put("magic", 42);
value.put("bool", true);
value.put("byte", (byte) 42);
value.put("short", (short) 42);
value.put("int", 42);
value.put("long", 42L);
value.put("float", 42f);
value.put("double", 42d);
value.put("string", "hmm");
value.put("date", new Date());
value.put("time", new Date());
value.put("timestamp", new Date());
value.put("decimal", new BigDecimal(42));
value.put("array", Arrays.asList(1, 2, 3));
value.put("map", Collections.singletonMap("what", "what"));
final List<String> maskFields = new ArrayList<>(schema.fields().size());
for (Field field: schema.fields()) {
if (!field.name().equals("magic")) {
maskFields.add(field.name());
}
}
final Struct updatedValue = (Struct) transform(maskFields).apply(record(schema, value)).value();
assertEquals(42, updatedValue.get("magic"));
assertEquals(false, updatedValue.get("bool"));
assertEquals((byte) 0, updatedValue.get("byte"));
assertEquals((short) 0, updatedValue.get("short"));
assertEquals(0, updatedValue.get("int"));
assertEquals(0L, updatedValue.get("long"));
assertEquals(0f, updatedValue.get("float"));
assertEquals(0d, updatedValue.get("double"));
assertEquals("", updatedValue.get("string"));
assertEquals(new Date(0), updatedValue.get("date"));
assertEquals(new Date(0), updatedValue.get("time"));
assertEquals(new Date(0), updatedValue.get("timestamp"));
assertEquals(BigDecimal.ZERO, updatedValue.get("decimal"));
assertEquals(Collections.emptyList(), updatedValue.get("array"));
assertEquals(Collections.emptyMap(), updatedValue.get("map"));
}
}

View File

@@ -0,0 +1,70 @@
/*
* 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.sink.SinkRecord;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
import static org.junit.Assert.assertEquals;
public class RegexRouterTest {
private static String apply(String regex, String replacement, String topic) {
final Map<String, String> props = new HashMap<>();
props.put("regex", regex);
props.put("replacement", replacement);
final RegexRouter<SinkRecord> router = new RegexRouter<>();
router.configure(props);
String sinkTopic = router.apply(new SinkRecord(topic, 0, null, null, null, null, 0)).topic();
router.close();
return sinkTopic;
}
@Test
public void staticReplacement() {
assertEquals("bar", apply("foo", "bar", "foo"));
}
@Test
public void doesntMatch() {
assertEquals("orig", apply("foo", "bar", "orig"));
}
@Test
public void identity() {
assertEquals("orig", apply("(.*)", "$1", "orig"));
}
@Test
public void addPrefix() {
assertEquals("prefix-orig", apply("(.*)", "prefix-$1", "orig"));
}
@Test
public void addSuffix() {
assertEquals("orig-suffix", apply("(.*)", "$1-suffix", "orig"));
}
@Test
public void slice() {
assertEquals("index", apply("(.*)-(\\d\\d\\d\\d\\d\\d\\d\\d)", "$1", "index-20160117"));
}
}

View File

@@ -0,0 +1,132 @@
/*
* 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.sink.SinkRecord;
import org.junit.After;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
public class ReplaceFieldTest {
private ReplaceField<SinkRecord> xform = new ReplaceField.Value<>();
@After
public void teardown() {
xform.close();
}
@Test
public void tombstoneSchemaless() {
final Map<String, String> props = new HashMap<>();
props.put("whitelist", "abc,foo");
props.put("renames", "abc:xyz,foo:bar");
xform.configure(props);
final SinkRecord record = new SinkRecord("test", 0, null, null, null, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertNull(transformedRecord.value());
assertNull(transformedRecord.valueSchema());
}
@Test
public void tombstoneWithSchema() {
final Map<String, String> props = new HashMap<>();
props.put("whitelist", "abc,foo");
props.put("renames", "abc:xyz,foo:bar");
xform.configure(props);
final Schema schema = SchemaBuilder.struct()
.field("dont", Schema.STRING_SCHEMA)
.field("abc", Schema.INT32_SCHEMA)
.field("foo", Schema.BOOLEAN_SCHEMA)
.field("etc", Schema.STRING_SCHEMA)
.build();
final SinkRecord record = new SinkRecord("test", 0, null, null, schema, null, 0);
final SinkRecord transformedRecord = xform.apply(record);
assertNull(transformedRecord.value());
assertEquals(schema, transformedRecord.valueSchema());
}
@Test
public void schemaless() {
final Map<String, String> props = new HashMap<>();
props.put("blacklist", "dont");
props.put("renames", "abc:xyz,foo:bar");
xform.configure(props);
final Map<String, Object> value = new HashMap<>();
value.put("dont", "whatever");
value.put("abc", 42);
value.put("foo", true);
value.put("etc", "etc");
final SinkRecord record = new SinkRecord("test", 0, null, null, null, value, 0);
final SinkRecord transformedRecord = xform.apply(record);
final Map updatedValue = (Map) transformedRecord.value();
assertEquals(3, updatedValue.size());
assertEquals(42, updatedValue.get("xyz"));
assertEquals(true, updatedValue.get("bar"));
assertEquals("etc", updatedValue.get("etc"));
}
@Test
public void withSchema() {
final Map<String, String> props = new HashMap<>();
props.put("whitelist", "abc,foo");
props.put("renames", "abc:xyz,foo:bar");
xform.configure(props);
final Schema schema = SchemaBuilder.struct()
.field("dont", Schema.STRING_SCHEMA)
.field("abc", Schema.INT32_SCHEMA)
.field("foo", Schema.BOOLEAN_SCHEMA)
.field("etc", Schema.STRING_SCHEMA)
.build();
final Struct value = new Struct(schema);
value.put("dont", "whatever");
value.put("abc", 42);
value.put("foo", true);
value.put("etc", "etc");
final SinkRecord record = new SinkRecord("test", 0, null, null, schema, value, 0);
final SinkRecord transformedRecord = xform.apply(record);
final Struct updatedValue = (Struct) transformedRecord.value();
assertEquals(2, updatedValue.schema().fields().size());
assertEquals(Integer.valueOf(42), updatedValue.getInt32("xyz"));
assertEquals(true, updatedValue.getBoolean("bar"));
}
}

View File

@@ -0,0 +1,149 @@
/*
* 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.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.sink.SinkRecord;
import org.junit.After;
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.assertSame;
public class SetSchemaMetadataTest {
private final SetSchemaMetadata<SinkRecord> xform = new SetSchemaMetadata.Value<>();
@After
public void teardown() {
xform.close();
}
@Test
public void schemaNameUpdate() {
xform.configure(Collections.singletonMap("schema.name", "foo"));
final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0);
final SinkRecord updatedRecord = xform.apply(record);
assertEquals("foo", updatedRecord.valueSchema().name());
}
@Test
public void schemaVersionUpdate() {
xform.configure(Collections.singletonMap("schema.version", 42));
final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0);
final SinkRecord updatedRecord = xform.apply(record);
assertEquals(Integer.valueOf(42), updatedRecord.valueSchema().version());
}
@Test
public void schemaNameAndVersionUpdate() {
final Map<String, String> props = new HashMap<>();
props.put("schema.name", "foo");
props.put("schema.version", "42");
xform.configure(props);
final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0);
final SinkRecord updatedRecord = xform.apply(record);
assertEquals("foo", updatedRecord.valueSchema().name());
assertEquals(Integer.valueOf(42), updatedRecord.valueSchema().version());
}
@Test
public void schemaNameAndVersionUpdateWithStruct() {
final String fieldName1 = "f1";
final String fieldName2 = "f2";
final String fieldValue1 = "value1";
final int fieldValue2 = 1;
final Schema schema = SchemaBuilder.struct()
.name("my.orig.SchemaDefn")
.field(fieldName1, Schema.STRING_SCHEMA)
.field(fieldName2, Schema.INT32_SCHEMA)
.build();
final Struct value = new Struct(schema).put(fieldName1, fieldValue1).put(fieldName2, fieldValue2);
final Map<String, String> props = new HashMap<>();
props.put("schema.name", "foo");
props.put("schema.version", "42");
xform.configure(props);
final SinkRecord record = new SinkRecord("", 0, null, null, schema, value, 0);
final SinkRecord updatedRecord = xform.apply(record);
assertEquals("foo", updatedRecord.valueSchema().name());
assertEquals(Integer.valueOf(42), updatedRecord.valueSchema().version());
// Make sure the struct's schema and fields all point to the new schema
assertMatchingSchema((Struct) updatedRecord.value(), updatedRecord.valueSchema());
}
@Test
public void updateSchemaOfStruct() {
final String fieldName1 = "f1";
final String fieldName2 = "f2";
final String fieldValue1 = "value1";
final int fieldValue2 = 1;
final Schema schema = SchemaBuilder.struct()
.name("my.orig.SchemaDefn")
.field(fieldName1, Schema.STRING_SCHEMA)
.field(fieldName2, Schema.INT32_SCHEMA)
.build();
final Struct value = new Struct(schema).put(fieldName1, fieldValue1).put(fieldName2, fieldValue2);
final Schema newSchema = SchemaBuilder.struct()
.name("my.updated.SchemaDefn")
.field(fieldName1, Schema.STRING_SCHEMA)
.field(fieldName2, Schema.INT32_SCHEMA)
.build();
Struct newValue = (Struct) SetSchemaMetadata.updateSchemaIn(value, newSchema);
assertMatchingSchema(newValue, newSchema);
}
@Test
public void updateSchemaOfNonStruct() {
Object value = Integer.valueOf(1);
Object updatedValue = SetSchemaMetadata.updateSchemaIn(value, Schema.INT32_SCHEMA);
assertSame(value, updatedValue);
}
@Test
public void updateSchemaOfNull() {
Object updatedValue = SetSchemaMetadata.updateSchemaIn(null, Schema.INT32_SCHEMA);
assertEquals(null, updatedValue);
}
protected void assertMatchingSchema(Struct value, Schema schema) {
assertSame(schema, value.schema());
assertEquals(schema.name(), value.schema().name());
for (Field field : schema.fields()) {
String fieldName = field.name();
assertEquals(schema.field(fieldName).name(), value.schema().field(fieldName).name());
assertEquals(schema.field(fieldName).index(), value.schema().field(fieldName).index());
assertSame(schema.field(fieldName).schema(), value.schema().field(fieldName).schema());
}
}
}

View File

@@ -0,0 +1,545 @@
/*
* 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.Date;
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.data.Time;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.After;
import org.junit.Test;
import java.util.Calendar;
import java.util.Collections;
import java.util.GregorianCalendar;
import java.util.HashMap;
import java.util.Map;
import java.util.TimeZone;
import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
public class TimestampConverterTest {
private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
private static final Calendar EPOCH;
private static final Calendar TIME;
private static final Calendar DATE;
private static final Calendar DATE_PLUS_TIME;
private static final long DATE_PLUS_TIME_UNIX;
private static final String STRING_DATE_FMT = "yyyy MM dd HH mm ss SSS z";
private static final String DATE_PLUS_TIME_STRING;
private final TimestampConverter<SourceRecord> xformKey = new TimestampConverter.Key<>();
private final TimestampConverter<SourceRecord> xformValue = new TimestampConverter.Value<>();
static {
EPOCH = GregorianCalendar.getInstance(UTC);
EPOCH.setTimeInMillis(0L);
TIME = GregorianCalendar.getInstance(UTC);
TIME.setTimeInMillis(0L);
TIME.add(Calendar.MILLISECOND, 1234);
DATE = GregorianCalendar.getInstance(UTC);
DATE.setTimeInMillis(0L);
DATE.set(1970, Calendar.JANUARY, 1, 0, 0, 0);
DATE.add(Calendar.DATE, 1);
DATE_PLUS_TIME = GregorianCalendar.getInstance(UTC);
DATE_PLUS_TIME.setTimeInMillis(0L);
DATE_PLUS_TIME.add(Calendar.DATE, 1);
DATE_PLUS_TIME.add(Calendar.MILLISECOND, 1234);
DATE_PLUS_TIME_UNIX = DATE_PLUS_TIME.getTime().getTime();
DATE_PLUS_TIME_STRING = "1970 01 02 00 00 01 234 UTC";
}
// Configuration
@After
public void teardown() {
xformKey.close();
xformValue.close();
}
@Test(expected = ConfigException.class)
public void testConfigNoTargetType() {
xformValue.configure(Collections.<String, String>emptyMap());
}
@Test(expected = ConfigException.class)
public void testConfigInvalidTargetType() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "invalid"));
}
@Test(expected = ConfigException.class)
public void testConfigMissingFormat() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "string"));
}
@Test(expected = ConfigException.class)
public void testConfigInvalidFormat() {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, "string");
config.put(TimestampConverter.FORMAT_CONFIG, "bad-format");
xformValue.configure(config);
}
// Conversions without schemas (most flexible Timestamp -> other types)
@Test
public void testSchemalessIdentity() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime()));
assertNull(transformed.valueSchema());
assertEquals(DATE_PLUS_TIME.getTime(), transformed.value());
}
@Test
public void testSchemalessTimestampToDate() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Date"));
SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime()));
assertNull(transformed.valueSchema());
assertEquals(DATE.getTime(), transformed.value());
}
@Test
public void testSchemalessTimestampToTime() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Time"));
SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime()));
assertNull(transformed.valueSchema());
assertEquals(TIME.getTime(), transformed.value());
}
@Test
public void testSchemalessTimestampToUnix() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "unix"));
SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime()));
assertNull(transformed.valueSchema());
assertEquals(DATE_PLUS_TIME_UNIX, transformed.value());
}
@Test
public void testSchemalessTimestampToString() {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, "string");
config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT);
xformValue.configure(config);
SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime()));
assertNull(transformed.valueSchema());
assertEquals(DATE_PLUS_TIME_STRING, transformed.value());
}
// Conversions without schemas (core types -> most flexible Timestamp format)
@Test
public void testSchemalessDateToTimestamp() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE.getTime()));
assertNull(transformed.valueSchema());
// No change expected since the source type is coarser-grained
assertEquals(DATE.getTime(), transformed.value());
}
@Test
public void testSchemalessTimeToTimestamp() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformValue.apply(createRecordSchemaless(TIME.getTime()));
assertNull(transformed.valueSchema());
// No change expected since the source type is coarser-grained
assertEquals(TIME.getTime(), transformed.value());
}
@Test
public void testSchemalessUnixToTimestamp() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME_UNIX));
assertNull(transformed.valueSchema());
assertEquals(DATE_PLUS_TIME.getTime(), transformed.value());
}
@Test
public void testSchemalessStringToTimestamp() {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp");
config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT);
xformValue.configure(config);
SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME_STRING));
assertNull(transformed.valueSchema());
assertEquals(DATE_PLUS_TIME.getTime(), transformed.value());
}
// Conversions with schemas (most flexible Timestamp -> other types)
@Test
public void testWithSchemaIdentity() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime()));
assertEquals(Timestamp.SCHEMA, transformed.valueSchema());
assertEquals(DATE_PLUS_TIME.getTime(), transformed.value());
}
@Test
public void testWithSchemaTimestampToDate() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Date"));
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime()));
assertEquals(Date.SCHEMA, transformed.valueSchema());
assertEquals(DATE.getTime(), transformed.value());
}
@Test
public void testWithSchemaTimestampToTime() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Time"));
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime()));
assertEquals(Time.SCHEMA, transformed.valueSchema());
assertEquals(TIME.getTime(), transformed.value());
}
@Test
public void testWithSchemaTimestampToUnix() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "unix"));
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime()));
assertEquals(Schema.INT64_SCHEMA, transformed.valueSchema());
assertEquals(DATE_PLUS_TIME_UNIX, transformed.value());
}
@Test
public void testWithSchemaTimestampToString() {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, "string");
config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT);
xformValue.configure(config);
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime()));
assertEquals(Schema.STRING_SCHEMA, transformed.valueSchema());
assertEquals(DATE_PLUS_TIME_STRING, transformed.value());
}
// Null-value conversions schemaless
@Test
public void testSchemalessNullValueToString() {
testSchemalessNullValueConversion("string");
testSchemalessNullFieldConversion("string");
}
@Test
public void testSchemalessNullValueToDate() {
testSchemalessNullValueConversion("Date");
testSchemalessNullFieldConversion("Date");
}
@Test
public void testSchemalessNullValueToTimestamp() {
testSchemalessNullValueConversion("Timestamp");
testSchemalessNullFieldConversion("Timestamp");
}
@Test
public void testSchemalessNullValueToUnix() {
testSchemalessNullValueConversion("unix");
testSchemalessNullFieldConversion("unix");
}
@Test
public void testSchemalessNullValueToTime() {
testSchemalessNullValueConversion("Time");
testSchemalessNullFieldConversion("Time");
}
private void testSchemalessNullValueConversion(String targetType) {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, targetType);
config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT);
xformValue.configure(config);
SourceRecord transformed = xformValue.apply(createRecordSchemaless(null));
assertNull(transformed.valueSchema());
assertNull(transformed.value());
}
private void testSchemalessNullFieldConversion(String targetType) {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, targetType);
config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT);
config.put(TimestampConverter.FIELD_CONFIG, "ts");
xformValue.configure(config);
SourceRecord transformed = xformValue.apply(createRecordSchemaless(null));
assertNull(transformed.valueSchema());
assertNull(transformed.value());
}
// Conversions with schemas (core types -> most flexible Timestamp format)
@Test
public void testWithSchemaDateToTimestamp() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Date.SCHEMA, DATE.getTime()));
assertEquals(Timestamp.SCHEMA, transformed.valueSchema());
// No change expected since the source type is coarser-grained
assertEquals(DATE.getTime(), transformed.value());
}
@Test
public void testWithSchemaTimeToTimestamp() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Time.SCHEMA, TIME.getTime()));
assertEquals(Timestamp.SCHEMA, transformed.valueSchema());
// No change expected since the source type is coarser-grained
assertEquals(TIME.getTime(), transformed.value());
}
@Test
public void testWithSchemaUnixToTimestamp() {
xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Schema.INT64_SCHEMA, DATE_PLUS_TIME_UNIX));
assertEquals(Timestamp.SCHEMA, transformed.valueSchema());
assertEquals(DATE_PLUS_TIME.getTime(), transformed.value());
}
@Test
public void testWithSchemaStringToTimestamp() {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp");
config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT);
xformValue.configure(config);
SourceRecord transformed = xformValue.apply(createRecordWithSchema(Schema.STRING_SCHEMA, DATE_PLUS_TIME_STRING));
assertEquals(Timestamp.SCHEMA, transformed.valueSchema());
assertEquals(DATE_PLUS_TIME.getTime(), transformed.value());
}
// Null-value conversions with schema
@Test
public void testWithSchemaNullValueToTimestamp() {
testWithSchemaNullValueConversion("Timestamp", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
testWithSchemaNullValueConversion("Timestamp", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
testWithSchemaNullValueConversion("Timestamp", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
testWithSchemaNullValueConversion("Timestamp", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
testWithSchemaNullValueConversion("Timestamp", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
}
@Test
public void testWithSchemaNullFieldToTimestamp() {
testWithSchemaNullFieldConversion("Timestamp", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
testWithSchemaNullFieldConversion("Timestamp", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
testWithSchemaNullFieldConversion("Timestamp", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
testWithSchemaNullFieldConversion("Timestamp", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
testWithSchemaNullFieldConversion("Timestamp", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA);
}
@Test
public void testWithSchemaNullValueToUnix() {
testWithSchemaNullValueConversion("unix", Schema.OPTIONAL_INT64_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
testWithSchemaNullValueConversion("unix", TimestampConverter.OPTIONAL_TIME_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
testWithSchemaNullValueConversion("unix", TimestampConverter.OPTIONAL_DATE_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
testWithSchemaNullValueConversion("unix", Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
testWithSchemaNullValueConversion("unix", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
}
@Test
public void testWithSchemaNullFieldToUnix() {
testWithSchemaNullFieldConversion("unix", Schema.OPTIONAL_INT64_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
testWithSchemaNullFieldConversion("unix", TimestampConverter.OPTIONAL_TIME_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
testWithSchemaNullFieldConversion("unix", TimestampConverter.OPTIONAL_DATE_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
testWithSchemaNullFieldConversion("unix", Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
testWithSchemaNullFieldConversion("unix", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA);
}
@Test
public void testWithSchemaNullValueToTime() {
testWithSchemaNullValueConversion("Time", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
testWithSchemaNullValueConversion("Time", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
testWithSchemaNullValueConversion("Time", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
testWithSchemaNullValueConversion("Time", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
testWithSchemaNullValueConversion("Time", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
}
@Test
public void testWithSchemaNullFieldToTime() {
testWithSchemaNullFieldConversion("Time", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
testWithSchemaNullFieldConversion("Time", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
testWithSchemaNullFieldConversion("Time", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
testWithSchemaNullFieldConversion("Time", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
testWithSchemaNullFieldConversion("Time", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA);
}
@Test
public void testWithSchemaNullValueToDate() {
testWithSchemaNullValueConversion("Date", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
testWithSchemaNullValueConversion("Date", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
testWithSchemaNullValueConversion("Date", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
testWithSchemaNullValueConversion("Date", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
testWithSchemaNullValueConversion("Date", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
}
@Test
public void testWithSchemaNullFieldToDate() {
testWithSchemaNullFieldConversion("Date", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
testWithSchemaNullFieldConversion("Date", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
testWithSchemaNullFieldConversion("Date", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
testWithSchemaNullFieldConversion("Date", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
testWithSchemaNullFieldConversion("Date", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA);
}
@Test
public void testWithSchemaNullValueToString() {
testWithSchemaNullValueConversion("string", Schema.OPTIONAL_INT64_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
testWithSchemaNullValueConversion("string", TimestampConverter.OPTIONAL_TIME_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
testWithSchemaNullValueConversion("string", TimestampConverter.OPTIONAL_DATE_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
testWithSchemaNullValueConversion("string", Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
testWithSchemaNullValueConversion("string", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
}
@Test
public void testWithSchemaNullFieldToString() {
testWithSchemaNullFieldConversion("string", Schema.OPTIONAL_INT64_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
testWithSchemaNullFieldConversion("string", TimestampConverter.OPTIONAL_TIME_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
testWithSchemaNullFieldConversion("string", TimestampConverter.OPTIONAL_DATE_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
testWithSchemaNullFieldConversion("string", Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
testWithSchemaNullFieldConversion("string", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
}
private void testWithSchemaNullValueConversion(String targetType, Schema originalSchema, Schema expectedSchema) {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, targetType);
config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT);
xformValue.configure(config);
SourceRecord transformed = xformValue.apply(createRecordWithSchema(originalSchema, null));
assertEquals(expectedSchema, transformed.valueSchema());
assertNull(transformed.value());
}
private void testWithSchemaNullFieldConversion(String targetType, Schema originalSchema, Schema expectedSchema) {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, targetType);
config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT);
config.put(TimestampConverter.FIELD_CONFIG, "ts");
xformValue.configure(config);
SchemaBuilder structSchema = SchemaBuilder.struct()
.field("ts", originalSchema)
.field("other", Schema.STRING_SCHEMA);
SchemaBuilder expectedStructSchema = SchemaBuilder.struct()
.field("ts", expectedSchema)
.field("other", Schema.STRING_SCHEMA);
Struct original = new Struct(structSchema);
original.put("ts", null);
original.put("other", "test");
// Struct field is null
SourceRecord transformed = xformValue.apply(createRecordWithSchema(structSchema.build(), original));
assertEquals(expectedStructSchema.build(), transformed.valueSchema());
assertNull(requireStruct(transformed.value(), "").get("ts"));
// entire Struct is null
transformed = xformValue.apply(createRecordWithSchema(structSchema.optional().build(), null));
assertEquals(expectedStructSchema.optional().build(), transformed.valueSchema());
assertNull(transformed.value());
}
// Convert field instead of entire key/value
@Test
public void testSchemalessFieldConversion() {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, "Date");
config.put(TimestampConverter.FIELD_CONFIG, "ts");
xformValue.configure(config);
Object value = Collections.singletonMap("ts", DATE_PLUS_TIME.getTime());
SourceRecord transformed = xformValue.apply(createRecordSchemaless(value));
assertNull(transformed.valueSchema());
assertEquals(Collections.singletonMap("ts", DATE.getTime()), transformed.value());
}
@Test
public void testWithSchemaFieldConversion() {
Map<String, String> config = new HashMap<>();
config.put(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp");
config.put(TimestampConverter.FIELD_CONFIG, "ts");
xformValue.configure(config);
// ts field is a unix timestamp
Schema structWithTimestampFieldSchema = SchemaBuilder.struct()
.field("ts", Schema.INT64_SCHEMA)
.field("other", Schema.STRING_SCHEMA)
.build();
Struct original = new Struct(structWithTimestampFieldSchema);
original.put("ts", DATE_PLUS_TIME_UNIX);
original.put("other", "test");
SourceRecord transformed = xformValue.apply(createRecordWithSchema(structWithTimestampFieldSchema, original));
Schema expectedSchema = SchemaBuilder.struct()
.field("ts", Timestamp.SCHEMA)
.field("other", Schema.STRING_SCHEMA)
.build();
assertEquals(expectedSchema, transformed.valueSchema());
assertEquals(DATE_PLUS_TIME.getTime(), ((Struct) transformed.value()).get("ts"));
assertEquals("test", ((Struct) transformed.value()).get("other"));
}
// Validate Key implementation in addition to Value
@Test
public void testKey() {
xformKey.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"));
SourceRecord transformed = xformKey.apply(new SourceRecord(null, null, "topic", 0, null, DATE_PLUS_TIME.getTime(), null, null));
assertNull(transformed.keySchema());
assertEquals(DATE_PLUS_TIME.getTime(), transformed.key());
}
private SourceRecord createRecordWithSchema(Schema schema, Object value) {
return new SourceRecord(null, null, "topic", 0, schema, value);
}
private SourceRecord createRecordSchemaless(Object value) {
return createRecordWithSchema(null, value);
}
}

View File

@@ -0,0 +1,48 @@
/*
* 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.source.SourceRecord;
import org.junit.After;
import org.junit.Test;
import java.util.Collections;
import static org.junit.Assert.assertEquals;
public class TimestampRouterTest {
private final TimestampRouter<SourceRecord> xform = new TimestampRouter<>();
@After
public void teardown() {
xform.close();
}
@Test
public void defaultConfiguration() {
xform.configure(Collections.<String, Object>emptyMap()); // defaults
final SourceRecord record = new SourceRecord(
null, null,
"test", 0,
null, null,
null, null,
1483425001864L
);
assertEquals("test-20170103", xform.apply(record).topic());
}
}

View File

@@ -0,0 +1,109 @@
/*
* 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.sink.SinkRecord;
import org.junit.After;
import org.junit.Test;
import java.util.Collections;
import java.util.HashMap;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
public class ValueToKeyTest {
private final ValueToKey<SinkRecord> xform = new ValueToKey<>();
@After
public void teardown() {
xform.close();
}
@Test
public void schemaless() {
xform.configure(Collections.singletonMap("fields", "a,b"));
final HashMap<String, Integer> value = new HashMap<>();
value.put("a", 1);
value.put("b", 2);
value.put("c", 3);
final SinkRecord record = new SinkRecord("", 0, null, null, null, value, 0);
final SinkRecord transformedRecord = xform.apply(record);
final HashMap<String, Integer> expectedKey = new HashMap<>();
expectedKey.put("a", 1);
expectedKey.put("b", 2);
assertNull(transformedRecord.keySchema());
assertEquals(expectedKey, transformedRecord.key());
}
@Test
public void withSchema() {
xform.configure(Collections.singletonMap("fields", "a,b"));
final Schema valueSchema = SchemaBuilder.struct()
.field("a", Schema.INT32_SCHEMA)
.field("b", Schema.INT32_SCHEMA)
.field("c", Schema.INT32_SCHEMA)
.build();
final Struct value = new Struct(valueSchema);
value.put("a", 1);
value.put("b", 2);
value.put("c", 3);
final SinkRecord record = new SinkRecord("", 0, null, null, valueSchema, value, 0);
final SinkRecord transformedRecord = xform.apply(record);
final Schema expectedKeySchema = SchemaBuilder.struct()
.field("a", Schema.INT32_SCHEMA)
.field("b", Schema.INT32_SCHEMA)
.build();
final Struct expectedKey = new Struct(expectedKeySchema)
.put("a", 1)
.put("b", 2);
assertEquals(expectedKeySchema, transformedRecord.keySchema());
assertEquals(expectedKey, transformedRecord.key());
}
@Test
public void nonExistingField() {
xform.configure(Collections.singletonMap("fields", "not_exist"));
final Schema valueSchema = SchemaBuilder.struct()
.field("a", Schema.INT32_SCHEMA)
.build();
final Struct value = new Struct(valueSchema);
value.put("a", 1);
final SinkRecord record = new SinkRecord("", 0, null, null, valueSchema, value, 0);
DataException actual = assertThrows(DataException.class, () -> xform.apply(record));
assertEquals("Field does not exist: not_exist", actual.getMessage());
}
}

View File

@@ -0,0 +1,40 @@
/*
* 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.util;
import org.apache.kafka.common.config.ConfigException;
import org.junit.Test;
import java.util.Collections;
public class NonEmptyListValidatorTest {
@Test(expected = ConfigException.class)
public void testNullList() {
new NonEmptyListValidator().ensureValid("foo", null);
}
@Test(expected = ConfigException.class)
public void testEmptyList() {
new NonEmptyListValidator().ensureValid("foo", Collections.emptyList());
}
@Test
public void testValidList() {
new NonEmptyListValidator().ensureValid("foo", Collections.singletonList("foo"));
}
}