Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,14 @@

import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.transforms.Cast;
import org.apache.kafka.connect.transforms.DropHeaders;
import org.apache.kafka.connect.transforms.ExtractField;
import org.apache.kafka.connect.transforms.Filter;
import org.apache.kafka.connect.transforms.Flatten;
import org.apache.kafka.connect.transforms.HeaderFrom;
import org.apache.kafka.connect.transforms.HoistField;
import org.apache.kafka.connect.transforms.InsertField;
import org.apache.kafka.connect.transforms.InsertHeader;
import org.apache.kafka.connect.transforms.MaskField;
import org.apache.kafka.connect.transforms.RegexRouter;
import org.apache.kafka.connect.transforms.ReplaceField;
Expand Down Expand Up @@ -62,7 +65,10 @@ private DocInfo(String transformationName, String overview, ConfigDef configDef)
new DocInfo(Flatten.class.getName(), Flatten.OVERVIEW_DOC, Flatten.CONFIG_DEF),
new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF),
new DocInfo(TimestampConverter.class.getName(), TimestampConverter.OVERVIEW_DOC, TimestampConverter.CONFIG_DEF),
new DocInfo(Filter.class.getName(), Filter.OVERVIEW_DOC, Filter.CONFIG_DEF)
new DocInfo(Filter.class.getName(), Filter.OVERVIEW_DOC, Filter.CONFIG_DEF),
new DocInfo(InsertHeader.class.getName(), InsertHeader.OVERVIEW_DOC, InsertHeader.CONFIG_DEF),
new DocInfo(DropHeaders.class.getName(), DropHeaders.OVERVIEW_DOC, DropHeaders.CONFIG_DEF),
new DocInfo(HeaderFrom.class.getName(), HeaderFrom.OVERVIEW_DOC, HeaderFrom.CONFIG_DEF)
);

private static void printTransformationHtml(PrintStream out, DocInfo docInfo) {
Expand Down
Original file line number Diff line number Diff line change
@@ -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.header.ConnectHeaders;
import org.apache.kafka.connect.header.Header;
import org.apache.kafka.connect.header.Headers;
import org.apache.kafka.connect.transforms.util.NonEmptyListValidator;
import org.apache.kafka.connect.transforms.util.SimpleConfig;

import java.util.HashSet;
import java.util.Map;
import java.util.Set;

import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE;

public class DropHeaders<R extends ConnectRecord<R>> implements Transformation<R> {

public static final String OVERVIEW_DOC =
"Removes one or more headers from each record.";

public static final String HEADERS_FIELD = "headers";

public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(HEADERS_FIELD, ConfigDef.Type.LIST,
NO_DEFAULT_VALUE, new NonEmptyListValidator(),
ConfigDef.Importance.HIGH,
"The name of the headers to be removed.");

private Set<String> headers;

@Override
public R apply(R record) {
Headers updatedHeaders = new ConnectHeaders();
for (Header header : record.headers()) {
if (!headers.contains(header.key())) {
updatedHeaders.add(header);
}
}
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(),
record.valueSchema(), record.value(), record.timestamp(), updatedHeaders);
}

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit, extra line

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

@Override
public void close() {
}

@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
headers = new HashSet<>(config.getList(HEADERS_FIELD));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,238 @@
/*
* 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.header.Header;
import org.apache.kafka.connect.header.Headers;
import org.apache.kafka.connect.transforms.util.NonEmptyListValidator;
import org.apache.kafka.connect.transforms.util.Requirements;
import org.apache.kafka.connect.transforms.util.SchemaUtil;
import org.apache.kafka.connect.transforms.util.SimpleConfig;

import java.util.HashMap;
import java.util.List;
import java.util.Map;

import static java.lang.String.format;
import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE;

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

public static final String FIELDS_FIELD = "fields";
public static final String HEADERS_FIELD = "headers";
public static final String OPERATION_FIELD = "operation";
private static final String MOVE_OPERATION = "move";
private static final String COPY_OPERATION = "copy";

public static final String OVERVIEW_DOC =
"Moves or copies fields in the key/value of a record into that record's headers. " +
"Corresponding elements of <code>" + FIELDS_FIELD + "</code> and " +
"<code>" + HEADERS_FIELD + "</code> together identify a field and the header it should be " +
"moved or copied to. " +
"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 ConfigDef CONFIG_DEF = new ConfigDef()
.define(FIELDS_FIELD, ConfigDef.Type.LIST,
NO_DEFAULT_VALUE, new NonEmptyListValidator(),
ConfigDef.Importance.HIGH,
"Field names in the record whose values are to be copied or moved to headers.")
.define(HEADERS_FIELD, ConfigDef.Type.LIST,
NO_DEFAULT_VALUE, new NonEmptyListValidator(),
ConfigDef.Importance.HIGH,
"Header names, in the same order as the field names listed in the fields configuration property.")
.define(OPERATION_FIELD, ConfigDef.Type.STRING, NO_DEFAULT_VALUE,
ConfigDef.ValidString.in(MOVE_OPERATION, COPY_OPERATION), ConfigDef.Importance.HIGH,
"Either <code>move</code> if the fields are to be moved to the headers (removed from the key/value), " +
"or <code>copy</code> if the fields are to be copied to the headers (retained in the key/value).");

enum Operation {
MOVE(MOVE_OPERATION),
COPY(COPY_OPERATION);

private final String name;

Operation(String name) {
this.name = name;
}

static Operation fromName(String name) {
switch (name) {
case MOVE_OPERATION:
return MOVE;
case COPY_OPERATION:
return COPY;
default:
throw new IllegalArgumentException();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Even though I don't think it's reachable by users, should we have a message here?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

It is impossible due to the ConfigDef.ValidString.in(MOVE_OPERATION, COPY_OPERATION), so this is really an assertion failure. The line number in the stacktrace would be enough to track it down if it ever did happen due to a later refactoring, so imho an error message is of no value. But I'm happy to add one if you like.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Ok, that's fair enough. Thanks

}
}

public String toString() {
return name;
}
}

private List<String> fields;

private List<String> headers;

private Operation operation;

private Cache<Schema, Schema> moveSchemaCache = new SynchronizedCache<>(new LRUCache<>(16));

@Override
public R apply(R record) {
Object operatingValue = operatingValue(record);
Schema operatingSchema = operatingSchema(record);

if (operatingSchema == null) {
return applySchemaless(record, operatingValue);
} else {
return applyWithSchema(record, operatingValue, operatingSchema);
}
}

private R applyWithSchema(R record, Object operatingValue, Schema operatingSchema) {
Headers updatedHeaders = record.headers().duplicate();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why duplicate headers here? According to the Header class's Javadocs, the collection should be mutable.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I wasn't completely sure whether a transformation was allowed to mutate headers, since it has to create a new ConnectRecord. Take this code from WorkerSourceTask as an example:

            final SourceRecord record = transformationChain.apply(preTransformRecord);
            final ProducerRecord<byte[], byte[]> producerRecord = convertTransformedRecord(record);
            if (producerRecord == null || retryWithToleranceOperator.failed()) {
                counter.skipRecord();
                commitTaskRecord(preTransformRecord, null);
                continue;
            }

See how preTransformRecord can be used after the transformation chain has been applied? I think it would be incorrect for commitTaskRecord to commit the original record but with headers which had been mutated by transformations, right?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Ah, that's fair. Does seem to be the pattern followed by the other out-of-the-box transformations as well; probably best to continue to follow that pattern.

I'm a little unnerved by this though, since as far as I can tell it's not publicly documented and so it's possible people writing their own transformations may be violating this implicit rule.

Out of scope, so I've filed KAFKA-10720 to track the need for possible documentation improvements.

Struct value = Requirements.requireStruct(operatingValue, "header " + operation);
final Schema updatedSchema;
final Struct updatedValue;
if (operation == Operation.MOVE) {
updatedSchema = moveSchema(operatingSchema);
updatedValue = new Struct(updatedSchema);
for (Field field : updatedSchema.fields()) {
updatedValue.put(field, value.get(field.name()));
}
} else {
updatedSchema = operatingSchema;
updatedValue = value;
}
for (int i = 0; i < fields.size(); i++) {
String fieldName = fields.get(i);
String headerName = headers.get(i);
Object fieldValue = value.schema().field(fieldName) != null ? value.get(fieldName) : null;
Schema fieldSchema = operatingSchema.field(fieldName).schema();
updatedHeaders.add(headerName, fieldValue, fieldSchema);
}
return newRecord(record, updatedSchema, updatedValue, updatedHeaders);
}

private Schema moveSchema(Schema operatingSchema) {
Schema moveSchema = this.moveSchemaCache.get(operatingSchema);
if (moveSchema == null) {
final SchemaBuilder builder = SchemaUtil.copySchemaBasics(operatingSchema, SchemaBuilder.struct());
for (Field field : operatingSchema.fields()) {
if (!fields.contains(field.name())) {
builder.field(field.name(), field.schema());
}
}
moveSchema = builder.build();
moveSchemaCache.put(operatingSchema, moveSchema);
}
return moveSchema;
}

private R applySchemaless(R record, Object operatingValue) {
Headers updatedHeaders = record.headers().duplicate();
Map<String, Object> value = Requirements.requireMap(operatingValue, "header " + operation);
Map<String, Object> updatedValue = new HashMap<>(value);
for (int i = 0; i < fields.size(); i++) {
String fieldName = fields.get(i);
Object fieldValue = value.get(fieldName);
String headerName = headers.get(i);
if (operation == Operation.MOVE) {
updatedValue.remove(fieldName);
}
updatedHeaders.add(headerName, fieldValue, null);
}
return newRecord(record, null, updatedValue, updatedHeaders);
}

protected abstract Object operatingValue(R record);
protected abstract Schema operatingSchema(R record);
protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue, Iterable<Header> updatedHeaders);

public static class Key<R extends ConnectRecord<R>> extends HeaderFrom<R> {

@Override
public Object operatingValue(R record) {
return record.key();
}

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

@Override
protected R newRecord(R record, Schema updatedSchema, Object updatedValue, Iterable<Header> updatedHeaders) {
return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue,
record.valueSchema(), record.value(), record.timestamp(), updatedHeaders);
}
}

public static class Value<R extends ConnectRecord<R>> extends HeaderFrom<R> {

@Override
public Object operatingValue(R record) {
return record.value();
}

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

@Override
protected R newRecord(R record, Schema updatedSchema, Object updatedValue, Iterable<Header> updatedHeaders) {
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(),
updatedSchema, updatedValue, record.timestamp(), updatedHeaders);
}
}

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

@Override
public void close() {

}

@Override
public void configure(Map<String, ?> props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
fields = config.getList(FIELDS_FIELD);
headers = config.getList(HEADERS_FIELD);
if (headers.size() != fields.size()) {
throw new ConfigException(format("'%s' config must have the same number of elements as '%s' config.",
FIELDS_FIELD, HEADERS_FIELD));
}
operation = Operation.fromName(config.getString(OPERATION_FIELD));
}
}
Loading