Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-3209: KIP-66: more single message transforms #2374

Closed
wants to merge 23 commits into from
Closed
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
@@ -0,0 +1,111 @@
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.Schema;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.transforms.util.SimpleConfig;

import java.util.Map;

public abstract class ExtractField<R extends ConnectRecord<R>> implements Transformation<R> {

public static final String FIELD_CONFIG = "field";

private 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 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);
final Object value = operatingValue(record);

if (schema == null) {
if (!(value instanceof Map)) {
throw new DataException("Only Map supported in schemaless mode");
}
return newRecord(record, null, ((Map) value).get(fieldName));
} else {
if (schema.type() != Schema.Type.STRUCT) {
throw new DataException("Only STRUCT schema type supported, was: " + schema.type());
}
return newRecord(record, schema.field(fieldName).schema(), ((Struct) 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());
}
}

}
Expand Up @@ -27,15 +27,16 @@
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 HoistToStruct<R extends ConnectRecord<R>> implements Transformation<R> {
public abstract class HoistField<R extends ConnectRecord<R>> implements Transformation<R> {

public static final String FIELD_CONFIG = "field";

private 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.");
"Field name for the single field that will be created in the resulting Struct or Map.");

private Cache<Schema, Schema> schemaUpdateCache;

Expand All @@ -53,15 +54,19 @@ public R apply(R record) {
final Schema schema = operatingSchema(record);
final Object value = operatingValue(record);

Schema updatedSchema = schemaUpdateCache.get(schema);
if (updatedSchema == null) {
updatedSchema = SchemaBuilder.struct().field(fieldName, schema).build();
schemaUpdateCache.put(schema, updatedSchema);
}
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);
final Struct updatedValue = new Struct(updatedSchema).put(fieldName, value);

return newRecord(record, updatedSchema, updatedValue);
return newRecord(record, updatedSchema, updatedValue);
}
}

@Override
Expand All @@ -81,10 +86,9 @@ public ConfigDef config() {
protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue);

/**
* Wraps the record key in a {@link org.apache.kafka.connect.data.Struct} with specified field name.
* Wraps the record key in a {@link Struct} when schema present, or a {@link Map} in schemaless mode, with the specified field name.
*/
public static class Key<R extends ConnectRecord<R>> extends HoistToStruct<R> {

public static class Key<R extends ConnectRecord<R>> extends HoistField<R> {
@Override
protected Schema operatingSchema(R record) {
return record.keySchema();
Expand All @@ -99,14 +103,12 @@ protected Object operatingValue(R record) {
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp());
}

}

/**
* Wraps the record value in a {@link org.apache.kafka.connect.data.Struct} with specified field name.
* Wraps the record value in a {@link Struct} when schema present, or a {@link Map} in schemaless mode, with the specified field name.
*/
public static class Value<R extends ConnectRecord<R>> extends HoistToStruct<R> {

public static class Value<R extends ConnectRecord<R>> extends HoistField<R> {
@Override
protected Schema operatingSchema(R record) {
return record.valueSchema();
Expand All @@ -121,7 +123,6 @@ protected Object operatingValue(R record) {
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp());
}

}

}
Expand Up @@ -37,7 +37,7 @@

public abstract class InsertField<R extends ConnectRecord<R>> implements Transformation<R> {

public interface Keys {
public interface ConfigName {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems this can be private instead? Ditto for other classes. I thought it was possibly for tests but it doesn't seem to be used anywhere anyway.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will do

String TOPIC_FIELD = "topic.field";
String PARTITION_FIELD = "partition.field";
String OFFSET_FIELD = "offset.field";
Expand All @@ -49,17 +49,17 @@ public interface Keys {
private static final String OPTIONALITY_DOC = "Suffix with '!' to make this a required field, or '?' to keep it optional (the default).";

private static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(Keys.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
.define(ConfigName.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka topic.\n" + OPTIONALITY_DOC)
.define(Keys.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
.define(ConfigName.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka partition.\n" + OPTIONALITY_DOC)
.define(Keys.OFFSET_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
.define(ConfigName.OFFSET_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka offset - only applicable to sink connectors.\n" + OPTIONALITY_DOC)
.define(Keys.TIMESTAMP_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
.define(ConfigName.TIMESTAMP_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for record timestamp.\n" + OPTIONALITY_DOC)
.define(Keys.STATIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
.define(ConfigName.STATIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for static data field.\n" + OPTIONALITY_DOC)
.define(Keys.STATIC_VALUE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
.define(ConfigName.STATIC_VALUE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Static field value, if field name configured.");

private static final Schema OPTIONAL_TIMESTAMP_SCHEMA = Timestamp.builder().optional().build();
Expand Down Expand Up @@ -98,12 +98,12 @@ public static InsertionSpec parse(String spec) {
@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
topicField = InsertionSpec.parse(config.getString(Keys.TOPIC_FIELD));
partitionField = InsertionSpec.parse(config.getString(Keys.PARTITION_FIELD));
offsetField = InsertionSpec.parse(config.getString(Keys.OFFSET_FIELD));
timestampField = InsertionSpec.parse(config.getString(Keys.TIMESTAMP_FIELD));
staticField = InsertionSpec.parse(config.getString(Keys.STATIC_FIELD));
staticValue = config.getString(Keys.STATIC_VALUE);
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);
applicable = topicField != null || partitionField != null || offsetField != null || timestampField != null;

schemaUpdateCache = new SynchronizedCache<>(new LRUCache<Schema, Schema>(16));
Expand Down
@@ -0,0 +1,117 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.ConnectSchema;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.transforms.util.SimpleConfig;

import java.util.Map;

public abstract class SetSchemaMetadata<R extends ConnectRecord<R>> implements Transformation<R> {

public interface ConfigName {
String SCHEMA_NAME = "schema.name";
String SCHEMA_VERSION = "schema.version";
}

private 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);
}

@Override
public R apply(R record) {
if (schemaName == null && schemaVersion == null) return record; // no-op
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In some cases like these I wonder if we should throw an exception during configure(). They probably messed something up if the transformation turns into a noop based only on configuration parameters.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's a good point. There are a few such transformations. I'll add an exception if the configuration is turning it into a no-op.

final Schema schema = operatingSchema(record);
if (schema == null) {
throw new DataException("Cannot update metadata on null Schema");
}
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,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you even need isMap and isArray? Shouldn't they already be null when they aren't maps or arrays?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These accessors throw an exception if the type doesn't line up...

isMap || isArray ? schema.valueSchema() : null
);
return updatedRecord(record, updatedSchema);
}

@Override
public ConfigDef config() {
return CONFIG_DEF;
}

@Override
public void close() {
}

protected abstract Schema operatingSchema(R record);

protected abstract R updatedRecord(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> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lack of multiple inheritance is really annoying in cases like this....

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Curious, how would multiple inheritance help?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

All these classes use the same pattern are defining the same overrides for operatingSchema and updatedRecord. If you put that into a single abstract class, then the definition of the Key and Value types would just be one line -- the class declaration that inherits from both SetSchemaMetadata and this new abstract class that provides the overrides.

It's obviously not a big deal here since the implementations are trivial, just kind of annoying code bloat.

@Override
protected Schema operatingSchema(R record) {
return record.keySchema();
}

@Override
protected R updatedRecord(R record, Schema updatedSchema) {
return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, record.key(), 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 updatedRecord(R record, Schema updatedSchema) {
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, record.value(), record.timestamp());
}
}

}