Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-3209: KIP-66: more single message transforms
Renames `HoistToStruct` SMT to `HoistField`. Adds the following SMTs: `ExtractField` `MaskField` `RegexRouter` `ReplaceField` `SetSchemaMetadata` `ValueToKey` Adds HTML doc generation and updates to `connect.html`. Author: Shikhar Bhushan <shikhar@confluent.io> Reviewers: Ewen Cheslack-Postava <ewen@confluent.io> Closes #2374 from shikhar/more-smt
- Loading branch information
Showing
24 changed files
with
1,779 additions
and
138 deletions.
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
87 changes: 87 additions & 0 deletions
87
connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.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,87 @@ | ||
/** | ||
* 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.tools; | ||
|
||
import org.apache.kafka.common.config.ConfigDef; | ||
import org.apache.kafka.connect.transforms.ExtractField; | ||
import org.apache.kafka.connect.transforms.HoistField; | ||
import org.apache.kafka.connect.transforms.InsertField; | ||
import org.apache.kafka.connect.transforms.MaskField; | ||
import org.apache.kafka.connect.transforms.RegexRouter; | ||
import org.apache.kafka.connect.transforms.ReplaceField; | ||
import org.apache.kafka.connect.transforms.SetSchemaMetadata; | ||
import org.apache.kafka.connect.transforms.TimestampRouter; | ||
import org.apache.kafka.connect.transforms.ValueToKey; | ||
|
||
import java.io.PrintStream; | ||
import java.util.Arrays; | ||
import java.util.List; | ||
|
||
public class TransformationDoc { | ||
|
||
private static final class DocInfo { | ||
final String transformationName; | ||
final String overview; | ||
final ConfigDef configDef; | ||
|
||
private DocInfo(String transformationName, String overview, ConfigDef configDef) { | ||
this.transformationName = transformationName; | ||
this.overview = overview; | ||
this.configDef = configDef; | ||
} | ||
} | ||
|
||
private static final List<DocInfo> TRANSFORMATIONS = Arrays.asList( | ||
new DocInfo(InsertField.class.getName(), InsertField.OVERVIEW_DOC, InsertField.CONFIG_DEF), | ||
new DocInfo(ReplaceField.class.getName(), ReplaceField.OVERVIEW_DOC, ReplaceField.CONFIG_DEF), | ||
new DocInfo(MaskField.class.getName(), MaskField.OVERVIEW_DOC, MaskField.CONFIG_DEF), | ||
new DocInfo(ValueToKey.class.getName(), ValueToKey.OVERVIEW_DOC, ValueToKey.CONFIG_DEF), | ||
new DocInfo(HoistField.class.getName(), HoistField.OVERVIEW_DOC, HoistField.CONFIG_DEF), | ||
new DocInfo(ExtractField.class.getName(), ExtractField.OVERVIEW_DOC, ExtractField.CONFIG_DEF), | ||
new DocInfo(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.CONFIG_DEF), | ||
new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF), | ||
new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF) | ||
); | ||
|
||
private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { | ||
out.println("<div id=\"" + docInfo.transformationName + "\">"); | ||
|
||
out.print("<h5>"); | ||
out.print(docInfo.transformationName); | ||
out.println("</h5>"); | ||
|
||
out.println(docInfo.overview); | ||
|
||
out.println("<p/>"); | ||
|
||
out.println(docInfo.configDef.toHtmlTable()); | ||
|
||
out.println("</div>"); | ||
} | ||
|
||
private static void printHtml(PrintStream out) throws NoSuchFieldException, IllegalAccessException, InstantiationException { | ||
for (final DocInfo docInfo : TRANSFORMATIONS) { | ||
printTransformationHtml(out, docInfo); | ||
} | ||
} | ||
|
||
public static void main(String... args) throws Exception { | ||
printHtml(System.out); | ||
} | ||
|
||
} |
114 changes: 114 additions & 0 deletions
114
connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.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,114 @@ | ||
/** | ||
* 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.transforms.util.SimpleConfig; | ||
|
||
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 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." | ||
+ "<p/>Use the concrete transformation type designed for the record key (<code>" + Key.class.getCanonicalName() + "</code>) " | ||
+ "or value (<code>" + Value.class.getCanonicalName() + "</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 = requireMap(operatingValue(record), PURPOSE); | ||
return newRecord(record, null, value.get(fieldName)); | ||
} else { | ||
final Struct value = requireStruct(operatingValue(record), PURPOSE); | ||
return newRecord(record, schema.field(fieldName).schema(), 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()); | ||
} | ||
} | ||
|
||
} |
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
Oops, something went wrong.