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
Closed
Changes from 1 commit
Commits
Show all changes
23 commits
Select commit
Hold shift + click to select a range
f34cc71
SetSchemaMetadata SMT
09af66b
Support schemaless and rename HoistToStruct->Hoist; add the inverse E…
022f492
InsertField transform -- fix bad name for interface containing config…
c5260a7
ValueToKey SMT
06b47a8
Clearer naming -- HoistField, ExtractField
cdd8855
Address review comments; docgen coming along
04dc9f3
Auto-generate docs/generated/connect_transforms.html
e90f989
html tweaks
692901f
consistent doc style
5772e95
Add RegexRouter SMT
2b99585
consistent naming
1063909
MaskField SMT
db7faf0
empty comment
b4aa9ee
Missed note in overview doc for MaskField SMT
3186c9f
Use Class object rather than canon name in masked value mappings
9bb5908
MaskField.fields -> maskedFields for clarity
8676dc5
Clearer overview doc for ValueToKey
16c9259
Config validation, try to fail-fast on no-op configs
a374bf9
checkstyle
7ee22e5
ReplaceField SMT and some consistency/cleanups refactoring
aa325db
Minor InsertField tweak -- docs and config check
be67396
User doc for transformations in connect.html
7eae01a
Better word choice
File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
230 changes: 230 additions & 0 deletions
230
connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,230 @@ | ||
/** | ||
* 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.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."; | ||
|
||
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() { | ||
@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<Schema, Schema>(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 (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()); | ||
} | ||
|
||
} | ||
|
||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This will probably grow outdated quickly. Can we use the classpath discovery code we already have to generate this list automatically?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, nm, I guess the issue is that we can get the class name and ConfigDef via the Transformation interface, but not the docstring. I wonder if a simple naming assumption and reflection could more cleanly solve this than manually maintaining the list.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah I was trying with reflection initially, however not sure how to handle
Key
/Value
variants since thePluginDiscovery
business only returns concrete impls currently, and I want to include a single entry for the SMT in these cases. It could be made to work but not sure if the magic is worth it.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Minor: I also like that the direct static dependency on
Xform.OVERVIEW_DOC
prevents it from appearing as an unused field in IntelliJ :)There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, if the gymnastics are too complicated, maintaining it manually seems fine. We probably won't need to change this very often anyway.