From 8f155350145c8a4edc0565e585b71293d9880d76 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 3 Jan 2017 11:21:17 -0800
Subject: [PATCH 01/36] KAFKA-3209: KIP-66: single message transforms
---
bin/kafka-run-class.sh | 2 +-
build.gradle | 42 +++-
checkstyle/import-control.xml | 6 +
.../apache/kafka/common/config/ConfigDef.java | 93 ++++----
.../connect/connector/ConnectRecord.java | 5 +-
.../apache/kafka/connect/sink/SinkRecord.java | 7 +-
.../kafka/connect/source/SourceRecord.java | 7 +-
.../connect/transforms/Transformation.java | 39 +++
.../kafka/connect/runtime/AbstractHerder.java | 12 +-
.../connect/runtime/ConnectorConfig.java | 223 +++++++++++++++++-
.../connect/runtime/TransformationChain.java | 61 +++++
.../apache/kafka/connect/runtime/Worker.java | 11 +-
.../kafka/connect/runtime/WorkerSinkTask.java | 21 +-
.../connect/runtime/WorkerSourceTask.java | 9 +-
.../connect/runtime/ConnectorConfigTest.java | 176 ++++++++++++++
.../connect/runtime/WorkerSinkTaskTest.java | 4 +-
.../runtime/WorkerSinkTaskThreadedTest.java | 2 +-
.../connect/runtime/WorkerSourceTaskTest.java | 2 +-
.../kafka/connect/runtime/WorkerTest.java | 3 +
.../kafka/connect/transforms/Insert.java | 213 +++++++++++++++++
.../kafka/connect/transforms/InsertInKey.java | 40 ++++
.../connect/transforms/InsertInValue.java | 40 ++++
.../connect/transforms/TimestampRouter.java | 86 +++++++
.../connect/transforms/InsertInValueTest.java | 88 +++++++
.../transforms/TimestampRouterTest.java | 43 ++++
settings.gradle | 2 +-
26 files changed, 1152 insertions(+), 85 deletions(-)
create mode 100644 connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java
create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java
create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java
diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh
index 41c75eb29ea7..af10f61b5c4c 100755
--- a/bin/kafka-run-class.sh
+++ b/bin/kafka-run-class.sh
@@ -111,7 +111,7 @@ do
CLASSPATH="$CLASSPATH:$dir/*"
done
-for cc_pkg in "api" "runtime" "file" "json" "tools"
+for cc_pkg in "api" "transforms" "runtime" "file" "json" "tools"
do
for file in "$base_dir"/connect/${cc_pkg}/build/libs/connect-${cc_pkg}*.jar;
do
diff --git a/build.gradle b/build.gradle
index 249cb2a7a0da..4d1c45b8aa5b 100644
--- a/build.gradle
+++ b/build.gradle
@@ -359,7 +359,7 @@ for ( sv in ['2_10', '2_11', '2_12'] ) {
}
}
-def connectPkgs = ['connect:api', 'connect:runtime', 'connect:json', 'connect:file']
+def connectPkgs = ['connect:api', 'connect:runtime', 'connect:transforms', 'connect:json', 'connect:file']
def pkgs = ['clients', 'examples', 'log4j-appender', 'tools', 'streams', 'streams:examples'] + connectPkgs
tasks.create(name: "jarConnect", dependsOn: connectPkgs.collect { it + ":jar" }) {}
@@ -530,6 +530,8 @@ project(':core') {
from(project(':connect:api').configurations.runtime) { into("libs/") }
from(project(':connect:runtime').jar) { into("libs/") }
from(project(':connect:runtime').configurations.runtime) { into("libs/") }
+ from(project(':connect:transforms').jar) { into("libs/") }
+ from(project(':connect:transforms').configurations.runtime) { into("libs/") }
from(project(':connect:json').jar) { into("libs/") }
from(project(':connect:json').configurations.runtime) { into("libs/") }
from(project(':connect:file').jar) { into("libs/") }
@@ -828,6 +830,42 @@ project(':connect:api') {
}
}
+project(':connect:transforms') {
+ archivesBaseName = "connect-transforms"
+
+ dependencies {
+ compile project(':connect:api')
+ compile libs.slf4jApi
+
+ testCompile libs.easymock
+ testCompile libs.junit
+ testCompile libs.powermock
+ testCompile libs.powermockEasymock
+
+ testRuntime libs.slf4jlog4j
+ }
+
+ javadoc {
+ enabled = false
+ }
+
+ tasks.create(name: "copyDependantLibs", type: Copy) {
+ from (configurations.testRuntime) {
+ include('slf4j-log4j12*')
+ }
+ from (configurations.runtime) {
+ exclude('kafka-clients*')
+ exclude('connect-*')
+ }
+ into "$buildDir/dependant-libs"
+ duplicatesStrategy 'exclude'
+ }
+
+ jar {
+ dependsOn copyDependantLibs
+ }
+}
+
project(':connect:json') {
archivesBaseName = "connect-json"
@@ -870,6 +908,7 @@ project(':connect:runtime') {
dependencies {
compile project(':connect:api')
+ compile project(":connect:transforms")
compile project(':clients')
compile project(':tools')
compile libs.slf4jApi
@@ -889,6 +928,7 @@ project(':connect:runtime') {
testCompile libs.junit
testCompile libs.powermock
testCompile libs.powermockEasymock
+
testCompile project(":connect:json")
testRuntime libs.slf4jlog4j
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 62cd77a20783..100f4f706c19 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -252,6 +252,12 @@
+
+
+
+
+
+
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
index 33f60a713389..feffb95f5814 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
@@ -70,10 +70,22 @@ public class ConfigDef {
public static final Object NO_DEFAULT_VALUE = new String("");
- private final Map configKeys = new HashMap<>();
- private final List groups = new LinkedList<>();
+ private final Map configKeys;
+ private final List groups;
private Set configsWithNoParent;
+ public ConfigDef() {
+ configKeys = new HashMap<>();
+ groups = new LinkedList<>();
+ configsWithNoParent = null;
+ }
+
+ public ConfigDef(ConfigDef base) {
+ configKeys = new HashMap<>(base.configKeys);
+ groups = new LinkedList<>(base.groups);
+ configsWithNoParent = base.configsWithNoParent;
+ }
+
/**
* Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef}
*
@@ -83,6 +95,17 @@ public Set names() {
return Collections.unmodifiableSet(configKeys.keySet());
}
+ public ConfigDef define(ConfigKey key) {
+ if (configKeys.containsKey(key.name)) {
+ throw new ConfigException("Configuration " + key.name + " is defined twice.");
+ }
+ if (key.group != null && !groups.contains(key.group)) {
+ groups.add(key.group);
+ }
+ configKeys.put(key.name, key);
+ return this;
+ }
+
/**
* Define a new configuration
* @param name the name of the config parameter
@@ -101,15 +124,7 @@ public Set names() {
*/
public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation,
String group, int orderInGroup, Width width, String displayName, List dependents, Recommender recommender) {
- if (configKeys.containsKey(name)) {
- throw new ConfigException("Configuration " + name + " is defined twice.");
- }
- if (group != null && !groups.contains(group)) {
- groups.add(group);
- }
- Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type);
- configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation, group, orderInGroup, width, displayName, dependents, recommender));
- return this;
+ return define(new ConfigKey(name, type, defaultValue, validator, importance, documentation, group, orderInGroup, width, displayName, dependents, recommender));
}
/**
@@ -584,7 +599,7 @@ private void validate(String name, Map parsed, Map dependents, Recommender recommender) {
this.name = name;
this.type = type;
- this.defaultValue = defaultValue;
+ this.defaultValue = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type);
this.validator = validator;
this.importance = importance;
- if (this.validator != null && this.hasDefault())
- this.validator.ensureValid(name, defaultValue);
+ if (this.validator != null && hasDefault())
+ this.validator.ensureValid(name, this.defaultValue);
this.documentation = documentation;
this.dependents = dependents;
this.group = group;
@@ -980,7 +995,7 @@ public String toEnrichedRst() {
StringBuilder b = new StringBuilder();
String lastKeyGroupName = "";
- for (ConfigKey def : sortedConfigsByGroup()) {
+ for (ConfigKey def : sortedConfigs()) {
if (def.group != null) {
if (!lastKeyGroupName.equalsIgnoreCase(def.group)) {
b.append(def.group).append("\n");
@@ -1033,39 +1048,12 @@ private void getConfigKeyRst(ConfigKey def, StringBuilder b) {
b.append(" * Importance: ").append(getConfigValue(def, "Importance")).append("\n");
}
- /**
- * Get a list of configs sorted into "natural" order: listing required fields first, then
- * ordering by importance, and finally by name.
- */
- protected List sortedConfigs() {
- // sort first required fields, then by importance, then name
- List configs = new ArrayList<>(this.configKeys.values());
- Collections.sort(configs, new Comparator() {
- public int compare(ConfigKey k1, ConfigKey k2) {
- // first take anything with no default value
- if (!k1.hasDefault() && k2.hasDefault()) {
- return -1;
- } else if (!k2.hasDefault() && k1.hasDefault()) {
- return 1;
- }
-
- // then sort by importance
- int cmp = k1.importance.compareTo(k2.importance);
- if (cmp == 0) {
- // then sort in alphabetical order
- return k1.name.compareTo(k2.name);
- } else {
- return cmp;
- }
- }
- });
- return configs;
- }
-
/**
* Get a list of configs sorted taking the 'group' and 'orderInGroup' into account.
+ *
+ * If grouping is not specified, the result will reflect "natural" order: listing required fields first, then ordering by importance, and finally by name.
*/
- protected List sortedConfigsByGroup() {
+ public List sortedConfigs() {
final Map groupOrd = new HashMap<>(groups.size());
int ord = 0;
for (String group: groups) {
@@ -1083,7 +1071,18 @@ public int compare(ConfigKey k1, ConfigKey k2) {
cmp = Integer.compare(k1.orderInGroup, k2.orderInGroup);
}
if (cmp == 0) {
- cmp = k1.name.compareTo(k2.name);
+ // first take anything with no default value
+ if (!k1.hasDefault() && k2.hasDefault()) {
+ cmp = -1;
+ } else if (!k2.hasDefault() && k1.hasDefault()) {
+ cmp = 1;
+ }
+ }
+ if (cmp == 0) {
+ cmp = k1.importance.compareTo(k2.importance);
+ }
+ if (cmp == 0) {
+ return k1.name.compareTo(k2.name);
}
return cmp;
}
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java
index d6319a171198..8b812bf3cd81 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java
@@ -29,7 +29,7 @@
*
*/
@InterfaceStability.Unstable
-public abstract class ConnectRecord {
+public abstract class ConnectRecord> {
private final String topic;
private final Integer kafkaPartition;
private final Schema keySchema;
@@ -79,6 +79,9 @@ public Long timestamp() {
return timestamp;
}
+ /** Generate a new record of the same type as itself, with the specified parameter values. **/
+ public abstract R newRecord(String topic, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp);
+
@Override
public String toString() {
return "ConnectRecord{" +
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java
index fbe1bdc24b74..ec88f139ea4d 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java
@@ -31,7 +31,7 @@
* timestamp, which may be {@code null}.
*/
@InterfaceStability.Unstable
-public class SinkRecord extends ConnectRecord {
+public class SinkRecord extends ConnectRecord {
private final long kafkaOffset;
private final TimestampType timestampType;
@@ -54,6 +54,11 @@ public TimestampType timestampType() {
return timestampType;
}
+ @Override
+ public SinkRecord newRecord(String topic, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp) {
+ return new SinkRecord(topic, kafkaPartition(), keySchema, key, valueSchema, value, kafkaOffset(), timestamp, timestampType);
+ }
+
@Override
public boolean equals(Object o) {
if (this == o)
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java
index 327b67bc94f8..4dbeeb529a5c 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java
@@ -42,7 +42,7 @@
*
*/
@InterfaceStability.Unstable
-public class SourceRecord extends ConnectRecord {
+public class SourceRecord extends ConnectRecord {
private final Map sourcePartition;
private final Map sourceOffset;
@@ -85,6 +85,11 @@ public SourceRecord(Map sourcePartition, Map sourceOffset,
return sourceOffset;
}
+ @Override
+ public SourceRecord newRecord(String topic, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp) {
+ return new SourceRecord(sourcePartition, sourceOffset, topic, kafkaPartition(), keySchema, key, valueSchema, value, timestamp);
+ }
+
@Override
public boolean equals(Object o) {
if (this == o)
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java b/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
new file mode 100644
index 000000000000..f0cde5dd27e6
--- /dev/null
+++ b/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
@@ -0,0 +1,39 @@
+/**
+ * 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 java.util.Map;
+
+public interface Transformation> {
+
+ /** Initialize with the provided configuration properties. **/
+ void init(Map props);
+
+ /** Apply transformation to the {@code record} and return another record object (which may be {@code record} itself). Must be thread-safe. **/
+ R apply(R record);
+
+ /** Signal that this transformation instance will no longer will be used. **/
+ void close();
+
+ /** Configuration specification for this transformation. **/
+ ConfigDef config();
+
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
index ba4894bc1e14..9f044a67f50b 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
@@ -251,12 +251,12 @@ public ConfigInfos validateConnectorConfig(Map connectorConfig)
throw new BadRequestException("Connector config " + connectorConfig + " contains no connector type");
Connector connector = getConnector(connType);
- ConfigDef connectorConfigDef;
- if (connector instanceof SourceConnector) {
- connectorConfigDef = SourceConnectorConfig.configDef();
- } else {
- connectorConfigDef = SinkConnectorConfig.configDef();
- }
+
+ final ConfigDef connectorConfigDef = ConnectorConfig.enrich(
+ (connector instanceof SourceConnector) ? SourceConnectorConfig.configDef() : SinkConnectorConfig.configDef(),
+ connectorConfig,
+ false
+ );
List configValues = new ArrayList<>();
Map configKeys = new HashMap<>();
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
index 30869a41d0be..f8e0596a5630 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
@@ -22,8 +22,19 @@
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigDef.Width;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.transforms.InsertInValue;
+import org.apache.kafka.connect.transforms.TimestampRouter;
+import org.apache.kafka.connect.transforms.Transformation;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
import java.util.Map;
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
@@ -41,6 +52,7 @@
*/
public class ConnectorConfig extends AbstractConfig {
protected static final String COMMON_GROUP = "Common";
+ protected static final String TRANSFORMS_GROUP = "Transforms";
public static final String NAME_CONFIG = "name";
private static final String NAME_DOC = "Globally unique name to use for this connector.";
@@ -48,7 +60,7 @@ public class ConnectorConfig extends AbstractConfig {
public static final String CONNECTOR_CLASS_CONFIG = "connector.class";
private static final String CONNECTOR_CLASS_DOC =
- "Name or alias of the class for this connector. Must be a subclass of org.apache.kafka.connect.connector.Connector. " +
+ "Name or alias of the class for this connector. Must be a subclass of org.apache.kafka.connect.connector.Connector. " +
"If the connector is org.apache.kafka.connect.file.FileStreamSinkConnector, you can either specify this full name, " +
" or use \"FileStreamSink\" or \"FileStreamSinkConnector\" to make the configuration a bit shorter";
private static final String CONNECTOR_CLASS_DISPLAY = "Connector class";
@@ -68,13 +80,27 @@ public class ConnectorConfig extends AbstractConfig {
private static final String TASK_MAX_DISPLAY = "Tasks max";
+ public static final String TRANSFORMS_CONFIG = "transforms";
+ private static final String TRANSFORMS_DOC = "Aliases for the transformations to be applied to records.";
+ private static final String TRANSFORMS_DISPLAY = "Transforms";
+
public static ConfigDef configDef() {
return new ConfigDef()
- .define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC, COMMON_GROUP, 1, Width.MEDIUM, NAME_DISPLAY)
- .define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC, COMMON_GROUP, 2, Width.LONG, CONNECTOR_CLASS_DISPLAY)
- .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, atLeast(TASKS_MIN_CONFIG), Importance.HIGH, TASKS_MAX_DOC, COMMON_GROUP, 3, Width.SHORT, TASK_MAX_DISPLAY)
- .define(KEY_CONVERTER_CLASS_CONFIG, Type.CLASS, null, Importance.LOW, KEY_CONVERTER_CLASS_DOC, COMMON_GROUP, 4, Width.SHORT, KEY_CONVERTER_CLASS_DISPLAY)
- .define(VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS, null, Importance.LOW, VALUE_CONVERTER_CLASS_DOC, COMMON_GROUP, 5, Width.SHORT, VALUE_CONVERTER_CLASS_DISPLAY);
+ .define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC, COMMON_GROUP, 1, Width.MEDIUM, NAME_DISPLAY)
+ .define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC, COMMON_GROUP, 2, Width.LONG, CONNECTOR_CLASS_DISPLAY)
+ .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, atLeast(TASKS_MIN_CONFIG), Importance.HIGH, TASKS_MAX_DOC, COMMON_GROUP, 3, Width.SHORT, TASK_MAX_DISPLAY)
+ .define(KEY_CONVERTER_CLASS_CONFIG, Type.CLASS, null, Importance.LOW, KEY_CONVERTER_CLASS_DOC, COMMON_GROUP, 4, Width.SHORT, KEY_CONVERTER_CLASS_DISPLAY)
+ .define(VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS, null, Importance.LOW, VALUE_CONVERTER_CLASS_DOC, COMMON_GROUP, 5, Width.SHORT, VALUE_CONVERTER_CLASS_DISPLAY)
+ .define(TRANSFORMS_CONFIG, Type.LIST, null, new ConfigDef.Validator() {
+ @Override
+ public void ensureValid(String name, Object value) {
+ if (value == null) return;
+ final List transformAliases = (List) value;
+ if (transformAliases.size() > new HashSet<>(transformAliases).size()) {
+ throw new ConfigException(name, value, "Duplicate alias provided.");
+ }
+ }
+ }, Importance.LOW, TRANSFORMS_DOC, TRANSFORMS_GROUP, 6, Width.LONG, TRANSFORMS_DISPLAY);
}
public ConnectorConfig() {
@@ -82,10 +108,189 @@ public ConnectorConfig() {
}
public ConnectorConfig(Map props) {
- super(configDef(), props);
+ this(configDef(), props);
+ }
+
+ public ConnectorConfig(ConfigDef configDef, Map props) {
+ super(enrich(configDef, props, true), props);
+ }
+
+ /**
+ * Returns the initialized list of {@link Transformation} which are specified in {@link #TRANSFORMS_CONFIG}.
+ */
+ public List transformations() {
+ final List transformAliases = getList(TRANSFORMS_CONFIG);
+ if (transformAliases == null || transformAliases.isEmpty()) {
+ return Collections.emptyList();
+ }
+
+ final List transformations = new ArrayList<>(transformAliases.size());
+ for (String alias : transformAliases) {
+ final String prefix = TRANSFORMS_CONFIG + "." + alias + ".";
+ final Transformation transformation;
+ try {
+ transformation = getClass(prefix + "type").asSubclass(Transformation.class).newInstance();
+ } catch (Exception e) {
+ throw new ConnectException(e);
+ }
+ transformation.init(originalsWithPrefix(prefix));
+ transformations.add(transformation);
+ }
+
+ return transformations;
+ }
+
+ /**
+ * Returns an enriched {@link ConfigDef} building upon the {@code ConfigDef}, using the current configuration specified in {@code props} as an input.
+ *
+ * {@code requireFullConfig} specifies whether required config values that are missing should cause an exception to be thrown.
+ */
+ public static ConfigDef enrich(ConfigDef baseConfigDef, Map props, boolean requireFullConfig) {
+ final List transformAliases = (List) ConfigDef.parseType(TRANSFORMS_CONFIG, props.get(TRANSFORMS_CONFIG), Type.LIST);
+ if (transformAliases == null || transformAliases.isEmpty()) {
+ return baseConfigDef;
+ }
+
+ final ConfigDef newDef = new ConfigDef(baseConfigDef);
+
+ for (String alias : new LinkedHashSet<>(transformAliases)) {
+ final String prefix = TRANSFORMS_CONFIG + "." + alias + ".";
+ final String groupPrefix = TRANSFORMS_GROUP + ": " + alias;
+ int orderInGroup = 0;
+
+ final String transformationTypeConfig = prefix + "type";
+ final ConfigDef.Validator typeValidator = new ConfigDef.Validator() {
+ @Override
+ public void ensureValid(String name, Object value) {
+ getConfigDefFromTransformation(transformationTypeConfig, (Class) value);
+ }
+ };
+ newDef.define(transformationTypeConfig, Type.CLASS, ConfigDef.NO_DEFAULT_VALUE, typeValidator, Importance.HIGH,
+ "Class for the '" + alias + "' transformation.", groupPrefix, orderInGroup++, Width.LONG, "Transformation type for " + alias,
+ Collections.emptyList(), new TransformationClassRecommender());
+
+ final ConfigDef transformationConfigDef;
+ try {
+ final String className = props.get(transformationTypeConfig);
+ final Class> cls = (Class>) ConfigDef.parseType(transformationTypeConfig, className, Type.CLASS);
+ transformationConfigDef = getConfigDefFromTransformation(transformationTypeConfig, cls);
+ } catch (ConfigException e) {
+ if (requireFullConfig) {
+ throw e;
+ } else {
+ continue;
+ }
+ }
+
+ for (ConfigDef.ConfigKey key : transformationConfigDef.sortedConfigs()) {
+ newDef.define(new ConfigDef.ConfigKey(
+ prefix + key.name,
+ key.type,
+ key.defaultValue,
+ embeddedValidator(prefix, key.validator),
+ key.importance,
+ key.documentation,
+ groupPrefix + (key.group == null ? "" : ": " + key.group),
+ orderInGroup++,
+ key.width,
+ key.displayName,
+ embeddedDependents(prefix, key.dependents),
+ embeddedRecommender(prefix, key.recommender)
+ ));
+ }
+ }
+
+ return newDef;
+ }
+
+ /**
+ * Return {@link ConfigDef} from {@code transformationCls}, which is expected to be a non-null {@code Class},
+ * by instantiating it and invoking {@link Transformation#config()}.
+ */
+ private static ConfigDef getConfigDefFromTransformation(String key, Class> transformationCls) {
+ if (transformationCls == null || !Transformation.class.isAssignableFrom(transformationCls)) {
+ throw new ConfigException(key, String.valueOf(transformationCls), "Not a Transformation");
+ }
+ try {
+ return (transformationCls.asSubclass(Transformation.class).newInstance()).config();
+ } catch (Exception e) {
+ throw new ConfigException(key, String.valueOf(transformationCls), "Error getting config definition from Transformation: " + e.getMessage());
+ }
+ }
+
+ /**
+ * Returns a new validator instance that delegates to the base validator but unprefixes the config name along the way.
+ */
+ private static ConfigDef.Validator embeddedValidator(final String prefix, final ConfigDef.Validator base) {
+ if (base == null) return null;
+ return new ConfigDef.Validator() {
+ @Override
+ public void ensureValid(String name, Object value) {
+ base.ensureValid(name.substring(prefix.length()), value);
+ }
+ };
+ }
+
+ /**
+ * Updated list of dependent configs with the specified {@code prefix} added.
+ */
+ private static List embeddedDependents(String prefix, List dependents) {
+ if (dependents == null) return null;
+ final List updatedDependents = new ArrayList<>(dependents.size());
+ for (String dependent : dependents) {
+ updatedDependents.add(prefix + dependent);
+ }
+ return updatedDependents;
}
- public ConnectorConfig(ConfigDef subClassConfig, Map props) {
- super(subClassConfig, props);
+ /**
+ * Returns a new recommender instance that delegates to the base recommender but unprefixes the input parameters along the way.
+ */
+ private static ConfigDef.Recommender embeddedRecommender(final String prefix, final ConfigDef.Recommender base) {
+ if (base == null) return null;
+ return new ConfigDef.Recommender() {
+ private String unprefixed(String k) {
+ return k.substring(prefix.length());
+ }
+
+ private Map unprefixed(Map parsedConfig) {
+ final Map unprefixedParsedConfig = new HashMap<>(parsedConfig.size());
+ for (Map.Entry e : parsedConfig.entrySet()) {
+ if (e.getKey().startsWith(prefix)) {
+ unprefixedParsedConfig.put(unprefixed(e.getKey()), e.getValue());
+ }
+ }
+ return unprefixedParsedConfig;
+ }
+
+ @Override
+ public List
+ *
+ * Note that it is not necessary that all records get a corresponding callback,
+ * as filtering is possible with a user-configured {@link org.apache.kafka.connect.transforms.Transformation}.
+ *
+ *
* @param record {@link SourceRecord} that was successfully sent via the producer.
* @throws InterruptedException
*/
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java b/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
index 81062c4e3417..7be3aa905c27 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
@@ -17,28 +17,27 @@
package org.apache.kafka.connect.transforms;
+import org.apache.kafka.common.Configurable;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.connector.ConnectRecord;
-import java.util.Map;
+import java.io.Closeable;
/**
* Single message transformation for Kafka Connect record types.
*
* Connectors can be configured with transformations to make lightweight message-at-a-time modifications.
*/
-public interface Transformation> {
-
- /** Initialize with the provided configuration properties. **/
- void init(Map props);
+public interface Transformation> extends Configurable, Closeable {
/** Apply transformation to the {@code record} and return another record object (which may be {@code record} itself). Must be thread-safe. **/
R apply(R record);
- /** Signal that this transformation instance will no longer will be used. **/
- void close();
-
/** Configuration specification for this transformation. **/
ConfigDef config();
+ /** Signal that this transformation instance will no longer will be used. **/
+ @Override
+ void close();
+
}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
index f86ffaa9fcb5..85bffaa0cc15 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
@@ -24,6 +24,9 @@
import org.apache.kafka.common.config.ConfigDef.Width;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.transforms.HoistKeyToStruct;
+import org.apache.kafka.connect.transforms.HoistValueToStruct;
+import org.apache.kafka.connect.transforms.InsertInKey;
import org.apache.kafka.connect.transforms.InsertInValue;
import org.apache.kafka.connect.transforms.TimestampRouter;
import org.apache.kafka.connect.transforms.Transformation;
@@ -133,7 +136,7 @@ public List transformations() {
} catch (Exception e) {
throw new ConnectException(e);
}
- transformation.init(originalsWithPrefix(prefix));
+ transformation.configure(originalsWithPrefix(prefix));
transformations.add(transformation);
}
@@ -162,18 +165,18 @@ public static ConfigDef enrich(ConfigDef baseConfigDef, Map prop
final ConfigDef.Validator typeValidator = new ConfigDef.Validator() {
@Override
public void ensureValid(String name, Object value) {
- TransformationConfig.getConfigDefFromTransformation(transformationTypeConfig, (Class) value);
+ getConfigDefFromTransformation(transformationTypeConfig, (Class) value);
}
};
newDef.define(transformationTypeConfig, Type.CLASS, ConfigDef.NO_DEFAULT_VALUE, typeValidator, Importance.HIGH,
"Class for the '" + alias + "' transformation.", groupPrefix, orderInGroup++, Width.LONG, "Transformation type for " + alias,
- Collections.emptyList(), new TransformationConfig.TransformationClassRecommender());
+ Collections.emptyList(), new TransformationClassRecommender());
final ConfigDef transformationConfigDef;
try {
final String className = props.get(transformationTypeConfig);
final Class> cls = (Class>) ConfigDef.parseType(transformationTypeConfig, className, Type.CLASS);
- transformationConfigDef = TransformationConfig.getConfigDefFromTransformation(transformationTypeConfig, cls);
+ transformationConfigDef = getConfigDefFromTransformation(transformationTypeConfig, cls);
} catch (ConfigException e) {
if (requireFullConfig) {
throw e;
@@ -182,119 +185,46 @@ public void ensureValid(String name, Object value) {
}
}
- for (ConfigDef.ConfigKey key : transformationConfigDef.sortedConfigs()) {
- newDef.define(new ConfigDef.ConfigKey(
- prefix + key.name,
- key.type,
- key.defaultValue,
- TransformationConfig.embeddedValidator(prefix, key.validator),
- key.importance,
- key.documentation,
- groupPrefix + (key.group == null ? "" : ": " + key.group),
- orderInGroup++,
- key.width,
- key.displayName,
- TransformationConfig.embeddedDependents(prefix, key.dependents),
- TransformationConfig.embeddedRecommender(prefix, key.recommender)
- ));
- }
+ newDef.embed(prefix, groupPrefix, orderInGroup, transformationConfigDef);
}
return newDef;
}
- private static final class TransformationConfig {
-
- /**
- * Return {@link ConfigDef} from {@code transformationCls}, which is expected to be a non-null {@code Class},
- * by instantiating it and invoking {@link Transformation#config()}.
- */
- static ConfigDef getConfigDefFromTransformation(String key, Class> transformationCls) {
- if (transformationCls == null || !Transformation.class.isAssignableFrom(transformationCls)) {
- throw new ConfigException(key, String.valueOf(transformationCls), "Not a Transformation");
- }
- try {
- return (transformationCls.asSubclass(Transformation.class).newInstance()).config();
- } catch (Exception e) {
- throw new ConfigException(key, String.valueOf(transformationCls), "Error getting config definition from Transformation: " + e.getMessage());
- }
- }
-
- /**
- * Returns a new validator instance that delegates to the base validator but unprefixes the config name along the way.
- */
- static ConfigDef.Validator embeddedValidator(final String prefix, final ConfigDef.Validator base) {
- if (base == null) return null;
- return new ConfigDef.Validator() {
- @Override
- public void ensureValid(String name, Object value) {
- base.ensureValid(name.substring(prefix.length()), value);
- }
- };
+ /**
+ * Return {@link ConfigDef} from {@code transformationCls}, which is expected to be a non-null {@code Class},
+ * by instantiating it and invoking {@link Transformation#config()}.
+ */
+ static ConfigDef getConfigDefFromTransformation(String key, Class> transformationCls) {
+ if (transformationCls == null || !Transformation.class.isAssignableFrom(transformationCls)) {
+ throw new ConfigException(key, String.valueOf(transformationCls), "Not a Transformation");
}
-
- /**
- * Updated list of dependent configs with the specified {@code prefix} added.
- */
- static List embeddedDependents(String prefix, List dependents) {
- if (dependents == null) return null;
- final List updatedDependents = new ArrayList<>(dependents.size());
- for (String dependent : dependents) {
- updatedDependents.add(prefix + dependent);
- }
- return updatedDependents;
+ try {
+ return (transformationCls.asSubclass(Transformation.class).newInstance()).config();
+ } catch (Exception e) {
+ throw new ConfigException(key, String.valueOf(transformationCls), "Error getting config definition from Transformation: " + e.getMessage());
}
+ }
- /**
- * Returns a new recommender instance that delegates to the base recommender but unprefixes the input parameters along the way.
- */
- static ConfigDef.Recommender embeddedRecommender(final String prefix, final ConfigDef.Recommender base) {
- if (base == null) return null;
- return new ConfigDef.Recommender() {
- private String unprefixed(String k) {
- return k.substring(prefix.length());
- }
-
- private Map unprefixed(Map parsedConfig) {
- final Map unprefixedParsedConfig = new HashMap<>(parsedConfig.size());
- for (Map.Entry e : parsedConfig.entrySet()) {
- if (e.getKey().startsWith(prefix)) {
- unprefixedParsedConfig.put(unprefixed(e.getKey()), e.getValue());
- }
- }
- return unprefixedParsedConfig;
- }
-
- @Override
- public List validValues(String name, Map parsedConfig) {
- return base.validValues(unprefixed(name), unprefixed(parsedConfig));
- }
-
- @Override
- public boolean visible(String name, Map parsedConfig) {
- return base.visible(unprefixed(name), unprefixed(parsedConfig));
- }
- };
+ /**
+ * Recommend bundled transformations.
+ */
+ static final class TransformationClassRecommender implements ConfigDef.Recommender {
+ @Override
+ public List validValues(String name, Map parsedConfig) {
+ return Arrays.asList(
+ HoistKeyToStruct.class,
+ HoistValueToStruct.class,
+ InsertInKey.class,
+ InsertInValue.class,
+ TimestampRouter.class
+ );
}
- /**
- * Recommend bundled transformations.
- */
- static final class TransformationClassRecommender implements ConfigDef.Recommender {
- @Override
- public List validValues(String name, Map parsedConfig) {
- return Arrays.asList(
- TimestampRouter.class,
- InsertInValue.class
- );
- }
-
- @Override
- public boolean visible(String name, Map parsedConfig) {
- return true;
- }
+ @Override
+ public boolean visible(String name, Map parsedConfig) {
+ return true;
}
-
}
}
\ No newline at end of file
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java
index c47fc360c08e..206b84e6ef82 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java
@@ -46,6 +46,12 @@ record = (R) transformation.apply(record);
return record;
}
+ public void close() {
+ for (Transformation transformation : transformations) {
+ transformation.close();
+ }
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) return true;
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
index 4a1184b8219b..b8532130e17f 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
@@ -131,6 +131,7 @@ protected void close() {
task.stop();
if (consumer != null)
consumer.close();
+ transformationChain.close();
}
@Override
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
index d8380247b3e9..1a4fda859adc 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
@@ -119,6 +119,7 @@ public void initialize(TaskConfig taskConfig) {
protected void close() {
producer.close(30, TimeUnit.SECONDS);
+ transformationChain.close();
}
@Override
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java
index 76e4f095ff03..bd4f512b0e9d 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java
@@ -42,7 +42,7 @@ public static class SimpleTransformation> implements
int magicNumber = 0;
@Override
- public void init(Map props) {
+ public void configure(Map props) {
magicNumber = Integer.parseInt((String) props.get("magic.number"));
}
@@ -80,34 +80,24 @@ public void danglingTransformAlias() {
new ConnectorConfig(props);
}
- @Test
+ @Test(expected = ConfigException.class)
public void wrongTransformationType() {
Map props = new HashMap<>();
props.put("name", "test");
props.put("connector.class", TestConnector.class.getName());
props.put("transforms", "a");
props.put("transforms.a.type", "uninstantiable");
- try {
- new ConnectorConfig(props);
- fail();
- } catch (ConfigException e) {
- assertTrue(e.getMessage().contains("Class uninstantiable could not be found"));
- }
+ new ConnectorConfig(props);
}
- @Test
+ @Test(expected = ConfigException.class)
public void unconfiguredTransform() {
Map props = new HashMap<>();
props.put("name", "test");
props.put("connector.class", TestConnector.class.getName());
props.put("transforms", "a");
props.put("transforms.a.type", SimpleTransformation.class.getName());
- try {
- new ConnectorConfig(props);
- fail();
- } catch (ConfigException e) {
- assertTrue(e.getMessage().contains("Missing required configuration"));
- }
+ new ConnectorConfig(props);
}
@Test
@@ -141,7 +131,7 @@ public void singleTransform() {
assertEquals(42, xform.magicNumber);
}
- @Test
+ @Test(expected = ConfigException.class)
public void multipleTransformsOneDangling() {
Map props = new HashMap<>();
props.put("name", "test");
@@ -149,11 +139,7 @@ public void multipleTransformsOneDangling() {
props.put("transforms", "a, b");
props.put("transforms.a.type", SimpleTransformation.class.getName());
props.put("transforms.a.magic.number", "42");
- try {
- new ConnectorConfig(props);
- } catch (ConfigException e) {
- assertTrue(e.getMessage().contains("Invalid value null for configuration transforms.b.type"));
- }
+ new ConnectorConfig(props);
}
@Test
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
index c2f5ca70d9ac..8a9ef79d373d 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
@@ -147,6 +147,9 @@ public Void answer() throws Throwable {
producer.close(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
EasyMock.expectLastCall();
+ transformationChain.close();
+ EasyMock.expectLastCall();
+
statusListener.onShutdown(taskId);
EasyMock.expectLastCall();
@@ -192,6 +195,9 @@ public void testPause() throws Exception {
producer.close(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
EasyMock.expectLastCall();
+ transformationChain.close();
+ EasyMock.expectLastCall();
+
PowerMock.replayAll();
workerTask.initialize(TASK_CONFIG);
@@ -238,6 +244,9 @@ public void testPollsInBackground() throws Exception {
producer.close(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
EasyMock.expectLastCall();
+ transformationChain.close();
+ EasyMock.expectLastCall();
+
PowerMock.replayAll();
workerTask.initialize(TASK_CONFIG);
@@ -283,6 +292,9 @@ public List answer() throws Throwable {
producer.close(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
EasyMock.expectLastCall();
+ transformationChain.close();
+ EasyMock.expectLastCall();
+
PowerMock.replayAll();
workerTask.initialize(TASK_CONFIG);
@@ -323,6 +335,9 @@ public void testCommit() throws Exception {
producer.close(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
EasyMock.expectLastCall();
+ transformationChain.close();
+ EasyMock.expectLastCall();
+
PowerMock.replayAll();
workerTask.initialize(TASK_CONFIG);
@@ -364,6 +379,9 @@ public void testCommitFailure() throws Exception {
producer.close(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
EasyMock.expectLastCall();
+ transformationChain.close();
+ EasyMock.expectLastCall();
+
PowerMock.replayAll();
workerTask.initialize(TASK_CONFIG);
@@ -510,6 +528,9 @@ public Object answer() throws Throwable {
producer.close(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
EasyMock.expectLastCall();
+ transformationChain.close();
+ EasyMock.expectLastCall();
+
PowerMock.replayAll();
workerTask.initialize(TASK_CONFIG);
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
index 1e5aa5ce9ba4..51d70121ba18 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
@@ -31,26 +31,18 @@
abstract class HoistToStruct> implements Transformation {
+ public static final String FIELD_CONFIG = "field";
+
private static final ConfigDef CONFIG_DEF = new ConfigDef()
- .define("field", ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM,
+ .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.");
- private static final class SchemaUpdateCacheEntry {
- final Schema base;
- final Schema updated;
-
- private SchemaUpdateCacheEntry(Schema base, Schema updated) {
- this.base = base;
- this.updated = updated;
- }
- }
-
private final SchemaUpdateCache schemaUpdateCache = new SchemaUpdateCache();
private String fieldName;
@Override
- public void init(Map props) {
+ public void configure(Map props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
fieldName = config.getString("field");
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
index 9c9a41a00cb5..7bfe451fbf28 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
@@ -23,27 +23,39 @@
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.sink.SinkRecord;
import org.apache.kafka.connect.transforms.util.SchemaUpdateCache;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
+import java.util.Date;
import java.util.Map;
abstract class Insert> implements Transformation {
- private static final String OPTIONALITY_DOC = "Prefix with '!' to make this a required field, or '?' to make it optional (the default).";
+ public enum Keys {
+ ;
+ public static final String TOPIC = "topic";
+ public static final String PARTITION = "partition";
+ public static final String OFFSET = "offset";
+ public static final String TIMESTAMP = "timestamp";
+ }
+
+ 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("topic", ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ .define(Keys.TOPIC, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka topic.\n" + OPTIONALITY_DOC)
- .define("partition", ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ .define(Keys.PARTITION, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka partition.\n" + OPTIONALITY_DOC)
- .define("offset", ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ .define(Keys.OFFSET, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka offset - only applicable to sink connectors.\n" + OPTIONALITY_DOC)
- .define("timestamp", ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ .define(Keys.TIMESTAMP, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for record timestamp.\n" + OPTIONALITY_DOC);
+ private static final Schema OPTIONAL_TIMESTAMP_SCHEMA = Timestamp.builder().optional().build();
+
private static final class InsertionSpec {
final String name;
final boolean optional;
@@ -55,26 +67,16 @@ private InsertionSpec(String name, boolean optional) {
public static InsertionSpec parse(String spec) {
if (spec == null) return null;
- if (spec.startsWith("?")) {
- return new InsertionSpec(spec.substring(1), true);
+ if (spec.endsWith("?")) {
+ return new InsertionSpec(spec.substring(0, spec.length() - 1), true);
}
- if (spec.startsWith("!")) {
- return new InsertionSpec(spec.substring(1), false);
+ if (spec.endsWith("!")) {
+ return new InsertionSpec(spec.substring(0, spec.length() - 1), false);
}
return new InsertionSpec(spec, true);
}
}
- private static final class SchemaUpdateCacheEntry {
- final Schema base;
- final Schema updated;
-
- private SchemaUpdateCacheEntry(Schema base, Schema updated) {
- this.base = base;
- this.updated = updated;
- }
- }
-
private final SchemaUpdateCache schemaUpdateCache = new SchemaUpdateCache();
private InsertionSpec topicField;
@@ -84,12 +86,12 @@ private SchemaUpdateCacheEntry(Schema base, Schema updated) {
private boolean applicable;
@Override
- public void init(Map props) {
+ public void configure(Map props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
- topicField = InsertionSpec.parse(config.getString("topic"));
- partitionField = InsertionSpec.parse(config.getString("partition"));
- offsetField = InsertionSpec.parse(config.getString("offset"));
- timestampField = InsertionSpec.parse(config.getString("timestamp"));
+ topicField = InsertionSpec.parse(config.getString(Keys.TOPIC));
+ partitionField = InsertionSpec.parse(config.getString(Keys.PARTITION));
+ offsetField = InsertionSpec.parse(config.getString(Keys.OFFSET));
+ timestampField = InsertionSpec.parse(config.getString(Keys.TIMESTAMP));
applicable = topicField != null || partitionField != null || offsetField != null || timestampField != null;
schemaUpdateCache.init();
@@ -126,17 +128,9 @@ public R apply(R record) {
private Schema makeUpdatedSchema(Schema schema) {
final SchemaBuilder builder = SchemaBuilder.struct();
- if (schema.name() != null) {
- builder.name(schema.name());
- }
-
- if (schema.version() != null) {
- builder.version(schema.version());
- }
-
- if (schema.doc() != null) {
- builder.doc(schema.doc());
- }
+ builder.name(schema.name());
+ builder.version(schema.version());
+ builder.doc(schema.doc());
final Map params = schema.parameters();
if (params != null) {
@@ -157,7 +151,7 @@ private Schema makeUpdatedSchema(Schema schema) {
builder.field(offsetField.name, offsetField.optional ? Schema.OPTIONAL_INT64_SCHEMA : Schema.INT64_SCHEMA);
}
if (timestampField != null) {
- builder.field(timestampField.name, timestampField.optional ? Schema.OPTIONAL_INT64_SCHEMA : Schema.INT64_SCHEMA);
+ builder.field(timestampField.name, timestampField.optional ? OPTIONAL_TIMESTAMP_SCHEMA : Timestamp.SCHEMA);
}
return builder.build();
@@ -178,12 +172,12 @@ private void insertFields(R record, Struct value) {
}
if (offsetField != null) {
if (!(record instanceof SinkRecord)) {
- throw new DataException("Offset insertion is only supported for sink conectors, record is of type: " + record.getClass());
+ throw new DataException("Offset insertion is only supported for sink connectors, record is of type: " + record.getClass());
}
value.put(offsetField.name, ((SinkRecord) record).kafkaOffset());
}
if (timestampField != null && record.timestamp() != null) {
- value.put(timestampField.name, record.timestamp());
+ value.put(timestampField.name, new Date(record.timestamp()));
}
}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
index 3ad755b8f0d3..62c638173b6b 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
@@ -30,27 +30,33 @@
/**
* This transformation facilitates updating the record's topic field as a function of the original topic value and the record timestamp.
*
- * The topic field determines what topic the records from a source connector are routed to.
- * In the case of a sink connector, the topic field is often used to determine the equivalent entity name in the destination system (e.g. database table name).
+ * It 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 class TimestampRouter> implements Transformation {
+ public enum Keys {
+ ;
+ public static final String TOPIC_FORMAT = "topic.format";
+ public static final String TIMESTAMP_FORMAT = "timestamp.format";
+ }
+
private static final ConfigDef CONFIG_DEF = new ConfigDef()
- .define("topic.format", ConfigDef.Type.STRING, "${topic}-${timestamp}", ConfigDef.Importance.HIGH,
+ .define(Keys.TOPIC_FORMAT, ConfigDef.Type.STRING, "${topic}-${timestamp}", ConfigDef.Importance.HIGH,
"Format string which can contain ``${topic}`` and ``${timestamp}`` as placeholders for the topic and timestamp, respectively.")
- .define("timestamp.format", ConfigDef.Type.STRING, "yyyyMMdd", ConfigDef.Importance.HIGH,
+ .define(Keys.TIMESTAMP_FORMAT, ConfigDef.Type.STRING, "yyyyMMdd", ConfigDef.Importance.HIGH,
"Format string for the timestamp that is compatible with java.text.SimpleDateFormat.");
private String topicFormat;
private ThreadLocal timestampFormat;
@Override
- public void init(Map props) {
+ public void configure(Map props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
- topicFormat = config.getString("topic.format");
+ topicFormat = config.getString(Keys.TOPIC_FORMAT);
- final String timestampFormatStr = config.getString("timestamp.format");
+ final String timestampFormatStr = config.getString(Keys.TIMESTAMP_FORMAT);
timestampFormat = new ThreadLocal() {
@Override
protected SimpleDateFormat initialValue() {
diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistKeyToStructTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistKeyToStructTest.java
index aef47d52cf3e..29aa0b41613b 100644
--- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistKeyToStructTest.java
+++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistKeyToStructTest.java
@@ -33,7 +33,7 @@ public class HoistKeyToStructTest {
@Test
public void sanityCheck() {
final HoistKeyToStruct xform = new HoistKeyToStruct<>();
- xform.init(Collections.singletonMap("field", "magic"));
+ 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);
diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
index 73f1f54b0453..3e442ac699a9 100644
--- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
+++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
@@ -20,6 +20,7 @@
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.Test;
@@ -36,7 +37,7 @@ public class InsertInValueTest {
@Test(expected = DataException.class)
public void schemaRequired() {
final InsertInValue xform = new InsertInValue<>();
- xform.init(Collections.singletonMap("topic", "topic_field"));
+ xform.configure(Collections.singletonMap("topic", "topic_field"));
xform.apply(new SourceRecord(null, null,
"", 0,
null, null));
@@ -45,7 +46,7 @@ public void schemaRequired() {
@Test(expected = DataException.class)
public void topLevelStructRequired() {
final InsertInValue xform = new InsertInValue<>();
- xform.init(Collections.singletonMap("topic", "topic_field"));
+ xform.configure(Collections.singletonMap("topic", "topic_field"));
xform.apply(new SourceRecord(null, null,
"", 0,
Schema.INT32_SCHEMA, 42));
@@ -54,12 +55,12 @@ public void topLevelStructRequired() {
@Test
public void copySchemaAndInsertConfiguredFields() {
final Map props = new HashMap<>();
- props.put("topic", "!topic_field");
+ props.put("topic", "topic_field!");
props.put("partition", "partition_field");
- props.put("timestamp", "?timestamp_field");
+ props.put("timestamp", "timestamp_field?");
final InsertInValue xform = new InsertInValue<>();
- xform.init(props);
+ xform.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);
@@ -80,7 +81,7 @@ public void copySchemaAndInsertConfiguredFields() {
assertEquals(Schema.OPTIONAL_INT32_SCHEMA, transformedRecord.valueSchema().field("partition_field").schema());
assertEquals(0, ((Struct) transformedRecord.value()).getInt32("partition_field").intValue());
- assertEquals(Schema.OPTIONAL_INT64_SCHEMA, transformedRecord.valueSchema().field("timestamp_field").schema());
+ assertEquals(Timestamp.builder().optional().build(), transformedRecord.valueSchema().field("timestamp_field").schema());
assertEquals(null, ((Struct) transformedRecord.value()).getInt64("timestamp_field"));
// Exercise caching
diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java
index b06ae13a61e5..cbd95a6eebaa 100644
--- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java
+++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java
@@ -29,7 +29,7 @@ public class TimestampRouterTest {
@Test
public void defaultConfiguration() {
final TimestampRouter xform = new TimestampRouter<>();
- xform.init(Collections.emptyMap()); // defaults
+ xform.configure(Collections.emptyMap()); // defaults
final SourceRecord record = new SourceRecord(
null, null,
"test", 0,
From df6008769ed75e83b791bcb60b9e09658beb3f97 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Fri, 6 Jan 2017 17:02:39 -0800
Subject: [PATCH 17/36] Add system test
---
.../tests/connect/connect_distributed_test.py | 63 +++++++++++++++++--
1 file changed, 59 insertions(+), 4 deletions(-)
diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py
index f49bb5d75381..db9191a0cc76 100644
--- a/tests/kafkatest/tests/connect/connect_distributed_test.py
+++ b/tests/kafkatest/tests/connect/connect_distributed_test.py
@@ -20,15 +20,16 @@
from ducktape.cluster.remoteaccount import RemoteCommandError
from kafkatest.services.zookeeper import ZookeeperService
-from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka import KafkaService, config_property
from kafkatest.services.connect import ConnectDistributedService, VerifiableSource, VerifiableSink, ConnectRestError, MockSink, MockSource
from kafkatest.services.console_consumer import ConsoleConsumer
from kafkatest.services.security.security_config import SecurityConfig
-import itertools, time
from collections import Counter, namedtuple
+import itertools
+import json
import operator
-
+import time
class ConnectDistributedTest(Test):
"""
@@ -70,10 +71,13 @@ def __init__(self, test_context):
self.value_converter = "org.apache.kafka.connect.json.JsonConverter"
self.schemas = True
- def setup_services(self, security_protocol=SecurityConfig.PLAINTEXT):
+ def setup_services(self, security_protocol=SecurityConfig.PLAINTEXT, timestamp_type=None):
self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk,
security_protocol=security_protocol, interbroker_security_protocol=security_protocol,
topics=self.topics)
+ if timestamp_type is not None:
+ for node in self.kafka.nodes:
+ node.config[config_property.MESSAGE_TIMESTAMP_TYPE] = timestamp_type
self.cc = ConnectDistributedService(self.test_context, 3, self.kafka, [self.INPUT_FILE, self.OUTPUT_FILE])
self.cc.log_level = "DEBUG"
@@ -434,6 +438,57 @@ def test_bounce(self, clean):
assert success, "Found validation errors:\n" + "\n ".join(errors)
+ @cluster(num_nodes=6)
+ def test_transformations(self):
+ self.setup_services(timestamp_type='CreateTime')
+ self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node))
+ self.cc.start()
+
+ start_time_ms = int(time.time() * 1000)
+ ts_fieldname = 'the_timestamp'
+
+ NamedConnector = namedtuple('Connector', ['name'])
+
+ source_connector = NamedConnector(name='file-src')
+
+ self.cc.create_connector({
+ 'name': source_connector.name,
+ 'connector.class': 'org.apache.kafka.connect.file.FileStreamSourceConnector',
+ 'tasks.max': 1,
+ 'file': self.INPUT_FILE,
+ 'topic': self.TOPIC,
+ 'transforms': 'hoistToStruct,insertTimestampField',
+ 'transforms.hoistToStruct.type': 'org.apache.kafka.connect.transforms.HoistValueToStruct',
+ 'transforms.hoistToStruct.field': 'content',
+ 'transforms.insertTimestampField.type': 'org.apache.kafka.connect.transforms.InsertInValue',
+ 'transforms.insertTimestampField.timestamp': ts_fieldname,
+ })
+
+ wait_until(lambda: self.connector_is_running(source_connector), timeout_sec=30, err_msg='Failed to see connector transition to the RUNNING state')
+
+ for node in self.cc.nodes:
+ node.account.ssh("echo -e -n " + repr(self.FIRST_INPUTS) + " >> " + self.INPUT_FILE)
+
+ consumer = ConsoleConsumer(self.test_context, 1, self.kafka, self.TOPIC, consumer_timeout_ms=15000)
+ consumer.run()
+
+ assert len(consumer.messages_consumed[1]) == len(self.FIRST_INPUT_LIST)
+
+ expected_schema = {
+ 'type': 'struct',
+ 'fields': [
+ {'field': 'content', 'type': 'string', 'optional': False},
+ {'field': ts_fieldname, 'name': 'org.apache.kafka.connect.data.Timestamp', 'type': 'int64', 'version': 1, 'optional': True},
+ ],
+ 'optional': False
+ }
+
+ for msg in consumer.messages_consumed[1]:
+ obj = json.loads(msg)
+ assert obj['schema'] == expected_schema
+ assert obj['payload']['content'] in self.FIRST_INPUT_LIST
+ assert obj['payload'][ts_fieldname] > start_time_ms
+
def _validate_file_output(self, input):
input_set = set(input)
# Output needs to be collected from all nodes because we can't be sure where the tasks will be scheduled.
From e0a6dc97b608302fad8695508289ead0d47740ac Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Mon, 9 Jan 2017 10:49:05 -0800
Subject: [PATCH 18/36] Schemaless support for InsertIn{Key,Value} transforms
---
.../kafka/connect/transforms/Insert.java | 41 ++++++++++++++++---
.../connect/transforms/InsertInValueTest.java | 21 ++++++++++
2 files changed, 57 insertions(+), 5 deletions(-)
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
index 7bfe451fbf28..542ab8db5541 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
@@ -30,6 +30,7 @@
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.Date;
+import java.util.HashMap;
import java.util.Map;
abstract class Insert> implements Transformation {
@@ -104,12 +105,42 @@ public R apply(R record) {
final Schema schema = operatingSchema(record);
final Object value = operatingValue(record);
- if (schema == null)
- throw new DataException("Schemaless mode not supported by " + getClass().getName());
+ if (value == null)
+ throw new DataException("null value");
+
+ if (schema == null) {
+ if (!(value instanceof Map))
+ throw new DataException("Can only operate on Map value in schemaless mode: " + value.getClass().getName());
+ return applySchemaless(record, (Map) value);
+ } else {
+ if (schema.type() != Schema.Type.STRUCT)
+ throw new DataException("Can only operate on Struct types: " + value.getClass().getName());
+ return applyWithSchema(record, schema, (Struct) value);
+ }
+ }
+
+ private R applySchemaless(R record, Map value) {
+ final Map 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) {
+ if (!(record instanceof SinkRecord))
+ throw new DataException("Offset insertion is only supported for sink connectors, record is of type: " + record.getClass());
+ updatedValue.put(offsetField.name, ((SinkRecord) record).kafkaOffset());
+ }
+ if (timestampField != null && record.timestamp() != null) {
+ updatedValue.put(timestampField.name, record.timestamp());
+ }
- if (schema.type() != Schema.Type.STRUCT)
- throw new DataException("Can only operate on Struct types: " + getClass().getName());
+ return newRecord(record, null, updatedValue);
+ }
+ private R applyWithSchema(R record, Schema schema, Struct value) {
Schema updatedSchema = schemaUpdateCache.get(schema);
if (updatedSchema == null) {
updatedSchema = makeUpdatedSchema(schema);
@@ -118,7 +149,7 @@ public R apply(R record) {
final Struct updatedValue = new Struct(updatedSchema);
- copyFields((Struct) value, updatedValue);
+ copyFields(value, updatedValue);
insertFields(record, updatedValue);
diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
index 3e442ac699a9..8c87d6c95625 100644
--- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
+++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
@@ -90,4 +90,25 @@ public void copySchemaAndInsertConfiguredFields() {
assertSame(transformedRecord.valueSchema(), transformedRecord2.valueSchema());
}
+ @Test
+ public void schemalessInsertConfiguredFields() {
+ final Map props = new HashMap<>();
+ props.put("topic", "topic_field!");
+ props.put("partition", "partition_field");
+ props.put("timestamp", "timestamp_field?");
+
+ final InsertInValue xform = new InsertInValue<>();
+ xform.configure(props);
+
+ final SourceRecord record = new SourceRecord(null, null, "test", 0,
+ null, Collections.singletonMap("magic", 42L));
+
+ final SourceRecord transformedRecord = xform.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"));
+ }
+
}
From d383c809320abaeef2d8eb3ac93ce2cf7ec835e0 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Mon, 9 Jan 2017 11:37:51 -0800
Subject: [PATCH 19/36] Allow adding static data in the InsertIn{Key,Value}
transform
---
.../kafka/connect/transforms/Insert.java | 46 +++++++++++++------
.../kafka/connect/transforms/InsertInKey.java | 1 +
.../connect/transforms/InsertInValue.java | 1 +
.../connect/transforms/InsertInValueTest.java | 31 ++++++-------
4 files changed, 49 insertions(+), 30 deletions(-)
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
index 542ab8db5541..cdbd940dc39f 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
@@ -37,23 +37,29 @@ abstract class Insert> implements Transformation {
public enum Keys {
;
- public static final String TOPIC = "topic";
- public static final String PARTITION = "partition";
- public static final String OFFSET = "offset";
- public static final String TIMESTAMP = "timestamp";
+ public static final String TOPIC_FIELD = "topic.field";
+ public static final String PARTITION_FIELD = "partition.field";
+ public static final String OFFSET_FIELD = "offset.field";
+ public static final String TIMESTAMP_FIELD = "timestamp.field";
+ public static final String STATIC_FIELD = "static.field";
+ public static final String STATIC_VALUE = "static.value";
}
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, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ .define(Keys.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka topic.\n" + OPTIONALITY_DOC)
- .define(Keys.PARTITION, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ .define(Keys.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
"Field name for Kafka partition.\n" + OPTIONALITY_DOC)
- .define(Keys.OFFSET, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ .define(Keys.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, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
- "Field name for record timestamp.\n" + OPTIONALITY_DOC);
+ .define(Keys.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,
+ "Field name for static data field.\n" + OPTIONALITY_DOC)
+ .define(Keys.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();
@@ -84,15 +90,19 @@ public static InsertionSpec parse(String spec) {
private InsertionSpec partitionField;
private InsertionSpec offsetField;
private InsertionSpec timestampField;
+ private InsertionSpec staticField;
+ private String staticValue;
private boolean applicable;
@Override
public void configure(Map props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
- topicField = InsertionSpec.parse(config.getString(Keys.TOPIC));
- partitionField = InsertionSpec.parse(config.getString(Keys.PARTITION));
- offsetField = InsertionSpec.parse(config.getString(Keys.OFFSET));
- timestampField = InsertionSpec.parse(config.getString(Keys.TIMESTAMP));
+ 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);
applicable = topicField != null || partitionField != null || offsetField != null || timestampField != null;
schemaUpdateCache.init();
@@ -136,7 +146,9 @@ private R applySchemaless(R record, Map value) {
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);
}
@@ -184,6 +196,9 @@ private Schema makeUpdatedSchema(Schema 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();
}
@@ -210,6 +225,9 @@ private void insertFields(R record, Struct value) {
if (timestampField != null && record.timestamp() != null) {
value.put(timestampField.name, new Date(record.timestamp()));
}
+ if (staticField != null && staticValue != null) {
+ value.put(staticField.name, staticValue);
+ }
}
@Override
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
index 5985cb6d109d..7003ca7c6bb0 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
@@ -22,6 +22,7 @@
/**
* This transformation allows inserting configured attributes of the record metadata as fields in the record key.
+ * It also allows adding a static data field.
* The record key is required to be of type {@link Schema.Type#STRUCT}.
*/
public class InsertInKey> extends Insert {
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
index 9c56d8fbd7c3..9aae2b714136 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
@@ -22,6 +22,7 @@
/**
* This transformation allows inserting configured attributes of the record metadata as fields in the record value.
+ * It also allows adding a static data field.
* The record value is required to be of type {@link Schema.Type#STRUCT}.
*/
public class InsertInValue> extends Insert {
diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
index 8c87d6c95625..e4408e886c00 100644
--- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
+++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertInValueTest.java
@@ -34,19 +34,10 @@
public class InsertInValueTest {
- @Test(expected = DataException.class)
- public void schemaRequired() {
- final InsertInValue xform = new InsertInValue<>();
- xform.configure(Collections.singletonMap("topic", "topic_field"));
- xform.apply(new SourceRecord(null, null,
- "", 0,
- null, null));
- }
-
@Test(expected = DataException.class)
public void topLevelStructRequired() {
final InsertInValue xform = new InsertInValue<>();
- xform.configure(Collections.singletonMap("topic", "topic_field"));
+ xform.configure(Collections.singletonMap("topic.field", "topic_field"));
xform.apply(new SourceRecord(null, null,
"", 0,
Schema.INT32_SCHEMA, 42));
@@ -55,9 +46,11 @@ public void topLevelStructRequired() {
@Test
public void copySchemaAndInsertConfiguredFields() {
final Map props = new HashMap<>();
- props.put("topic", "topic_field!");
- props.put("partition", "partition_field");
- props.put("timestamp", "timestamp_field?");
+ 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");
final InsertInValue xform = new InsertInValue<>();
xform.configure(props);
@@ -84,6 +77,9 @@ public void copySchemaAndInsertConfiguredFields() {
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 = xform.apply(
new SourceRecord(null, null, "test", 1, simpleStructSchema, new Struct(simpleStructSchema)));
@@ -93,9 +89,11 @@ public void copySchemaAndInsertConfiguredFields() {
@Test
public void schemalessInsertConfiguredFields() {
final Map props = new HashMap<>();
- props.put("topic", "topic_field!");
- props.put("partition", "partition_field");
- props.put("timestamp", "timestamp_field?");
+ 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");
final InsertInValue xform = new InsertInValue<>();
xform.configure(props);
@@ -109,6 +107,7 @@ public void schemalessInsertConfiguredFields() {
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"));
}
}
From f5d151e716b1d4faf958c153d8253516c5a3a7fd Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Mon, 9 Jan 2017 13:47:53 -0800
Subject: [PATCH 20/36] Factor out classpath-scanning plugin discovery business
so can use it for transformations
---
.../kafka/connect/runtime/AbstractHerder.java | 41 +------
.../connect/runtime/ConnectorConfig.java | 14 +--
.../connect/runtime/PluginDiscovery.java | 111 ++++++++++++++++++
.../resources/ConnectorPluginsResource.java | 3 +-
4 files changed, 115 insertions(+), 54 deletions(-)
create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
index 9f044a67f50b..4bb68c078577 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
@@ -33,17 +33,7 @@
import org.apache.kafka.connect.source.SourceConnector;
import org.apache.kafka.connect.storage.ConfigBackingStore;
import org.apache.kafka.connect.storage.StatusBackingStore;
-import org.apache.kafka.connect.tools.MockConnector;
-import org.apache.kafka.connect.tools.MockSinkConnector;
-import org.apache.kafka.connect.tools.MockSourceConnector;
-import org.apache.kafka.connect.tools.SchemaSourceConnector;
-import org.apache.kafka.connect.tools.VerifiableSinkConnector;
-import org.apache.kafka.connect.tools.VerifiableSourceConnector;
import org.apache.kafka.connect.util.ConnectorTaskId;
-import org.apache.kafka.connect.util.ReflectionsUtil;
-import org.reflections.Reflections;
-import org.reflections.util.ClasspathHelper;
-import org.reflections.util.ConfigurationBuilder;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
@@ -89,13 +79,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
protected final ConfigBackingStore configBackingStore;
private Map tempConnectors = new ConcurrentHashMap<>();
- private static List validConnectorPlugins;
- private static final Object LOCK = new Object();
private Thread classPathTraverser;
- private static final List> EXCLUDES = Arrays.asList(
- VerifiableSourceConnector.class, VerifiableSinkConnector.class,
- MockConnector.class, MockSourceConnector.class, MockSinkConnector.class,
- SchemaSourceConnector.class);
public AbstractHerder(Worker worker,
String workerId,
@@ -278,29 +262,6 @@ public ConfigInfos validateConnectorConfig(Map connectorConfig)
return generateResult(connType, configKeys, configValues, allGroups);
}
- public static List connectorPlugins() {
- synchronized (LOCK) {
- if (validConnectorPlugins != null) {
- return validConnectorPlugins;
- }
- ReflectionsUtil.registerUrlTypes();
- ConfigurationBuilder builder = new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath());
- Reflections reflections = new Reflections(builder);
-
- Set> connectorClasses = reflections.getSubTypesOf(Connector.class);
- connectorClasses.removeAll(EXCLUDES);
- List connectorPlugins = new LinkedList<>();
- for (Class extends Connector> connectorClass : connectorClasses) {
- int mod = connectorClass.getModifiers();
- if (!Modifier.isAbstract(mod) && !Modifier.isInterface(mod)) {
- connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName()));
- }
- }
- validConnectorPlugins = connectorPlugins;
- return connectorPlugins;
- }
- }
-
// public for testing
public static ConfigInfos generateResult(String connType, Map configKeys, List configValues, List groups) {
int errorCount = 0;
@@ -394,7 +355,7 @@ private void traverseClassPath() {
classPathTraverser = new Thread(new Runnable() {
@Override
public void run() {
- connectorPlugins();
+ PluginDiscovery.scanClasspathForPlugins();
}
}, "CLASSPATH traversal thread.");
classPathTraverser.start();
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
index 85bffaa0cc15..91790a3173c9 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
@@ -24,15 +24,9 @@
import org.apache.kafka.common.config.ConfigDef.Width;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.connect.errors.ConnectException;
-import org.apache.kafka.connect.transforms.HoistKeyToStruct;
-import org.apache.kafka.connect.transforms.HoistValueToStruct;
-import org.apache.kafka.connect.transforms.InsertInKey;
-import org.apache.kafka.connect.transforms.InsertInValue;
-import org.apache.kafka.connect.transforms.TimestampRouter;
import org.apache.kafka.connect.transforms.Transformation;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@@ -212,13 +206,7 @@ static ConfigDef getConfigDefFromTransformation(String key, Class> transformat
static final class TransformationClassRecommender implements ConfigDef.Recommender {
@Override
public List validValues(String name, Map parsedConfig) {
- return Arrays.asList(
- HoistKeyToStruct.class,
- HoistValueToStruct.class,
- InsertInKey.class,
- InsertInValue.class,
- TimestampRouter.class
- );
+ return (List) PluginDiscovery.transformationPlugins();
}
@Override
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
new file mode 100644
index 000000000000..518715683f84
--- /dev/null
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
@@ -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
+ *
+ * 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.runtime;
+
+import org.apache.kafka.connect.connector.Connector;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo;
+import org.apache.kafka.connect.tools.MockConnector;
+import org.apache.kafka.connect.tools.MockSinkConnector;
+import org.apache.kafka.connect.tools.MockSourceConnector;
+import org.apache.kafka.connect.tools.SchemaSourceConnector;
+import org.apache.kafka.connect.tools.VerifiableSinkConnector;
+import org.apache.kafka.connect.tools.VerifiableSourceConnector;
+import org.apache.kafka.connect.transforms.Transformation;
+import org.apache.kafka.connect.util.ReflectionsUtil;
+import org.reflections.Reflections;
+import org.reflections.util.ClasspathHelper;
+import org.reflections.util.ConfigurationBuilder;
+
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+public class PluginDiscovery {
+
+ private static final List> CONNECTOR_EXCLUDES = Arrays.asList(
+ VerifiableSourceConnector.class, VerifiableSinkConnector.class,
+ MockConnector.class, MockSourceConnector.class, MockSinkConnector.class,
+ SchemaSourceConnector.class
+ );
+
+ private static final List> TRANSFORMATION_EXCLUDES = Arrays.asList();
+
+ private static boolean scanned = false;
+ private static List validConnectorPlugins;
+ private static List> validTransformationPlugins;
+
+ public static synchronized void scanClasspathForPlugins() {
+ if (scanned) return;
+
+ ReflectionsUtil.registerUrlTypes();
+ final Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath()));
+
+ {
+ final Set> connectorClasses = reflections.getSubTypesOf(Connector.class);
+ connectorClasses.removeAll(CONNECTOR_EXCLUDES);
+
+ final List connectorPlugins = new LinkedList<>();
+ for (Class extends Connector> connectorClass : connectorClasses) {
+ if (isConcrete(connectorClass)) {
+ connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName()));
+ }
+ }
+
+ validConnectorPlugins = connectorPlugins;
+ }
+
+ {
+ final Set> transformationClasses = reflections.getSubTypesOf(Transformation.class);
+ transformationClasses.removeAll(TRANSFORMATION_EXCLUDES);
+
+ final List> transformationPlugins = new LinkedList<>();
+ for (Class extends Transformation> transformationClass : transformationClasses) {
+ if (isConcrete(transformationClass)) {
+ transformationPlugins.add(transformationClass);
+ }
+ }
+
+ validTransformationPlugins = transformationPlugins;
+ }
+
+ scanned = true;
+ }
+
+ public static synchronized List connectorPlugins() {
+ scanClasspathForPlugins();
+ return validConnectorPlugins;
+ }
+
+ public static synchronized List> transformationPlugins() {
+ scanClasspathForPlugins();
+ return validTransformationPlugins;
+ }
+
+ private static boolean isConcrete(Class> cls) {
+ final int mod = cls.getModifiers();
+ return !Modifier.isAbstract(mod) && !Modifier.isInterface(mod);
+ }
+
+ public static void main(String... args) {
+ System.out.println(connectorPlugins());
+ System.out.println(transformationPlugins());
+ }
+
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java
index 519aa9a747df..1b56bc58531c 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java
@@ -20,6 +20,7 @@
import org.apache.kafka.connect.runtime.AbstractHerder;
import org.apache.kafka.connect.runtime.ConnectorConfig;
import org.apache.kafka.connect.runtime.Herder;
+import org.apache.kafka.connect.runtime.PluginDiscovery;
import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo;
@@ -60,6 +61,6 @@ public ConfigInfos validateConfigs(final @PathParam("connectorType") String conn
@GET
@Path("/")
public List listConnectorPlugins() {
- return AbstractHerder.connectorPlugins();
+ return PluginDiscovery.connectorPlugins();
}
}
From a557c68e41e169ae7132c4cb78fcb1a626254193 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Mon, 9 Jan 2017 14:22:24 -0800
Subject: [PATCH 21/36] PluginDiscovery -- sort the collections
---
.../connect/runtime/PluginDiscovery.java | 78 +++++++++++--------
1 file changed, 47 insertions(+), 31 deletions(-)
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
index 518715683f84..ccea6660c40d 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
@@ -33,6 +33,8 @@
import java.lang.reflect.Modifier;
import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
@@ -51,51 +53,65 @@ public class PluginDiscovery {
private static List validConnectorPlugins;
private static List> validTransformationPlugins;
+ public static synchronized List connectorPlugins() {
+ scanClasspathForPlugins();
+ return validConnectorPlugins;
+ }
+
+ public static synchronized List> transformationPlugins() {
+ scanClasspathForPlugins();
+ return validTransformationPlugins;
+ }
+
public static synchronized void scanClasspathForPlugins() {
if (scanned) return;
-
ReflectionsUtil.registerUrlTypes();
final Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath()));
+ validConnectorPlugins = Collections.unmodifiableList(connectorPlugins(reflections));
+ validTransformationPlugins = Collections.unmodifiableList(transformationPlugins(reflections));
+ scanned = true;
+ }
- {
- final Set> connectorClasses = reflections.getSubTypesOf(Connector.class);
- connectorClasses.removeAll(CONNECTOR_EXCLUDES);
+ private static List connectorPlugins(Reflections reflections) {
+ final Set> connectorClasses = reflections.getSubTypesOf(Connector.class);
+ connectorClasses.removeAll(CONNECTOR_EXCLUDES);
- final List connectorPlugins = new LinkedList<>();
- for (Class extends Connector> connectorClass : connectorClasses) {
- if (isConcrete(connectorClass)) {
- connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName()));
- }
+ final List connectorPlugins = new LinkedList<>();
+ for (Class extends Connector> connectorClass : connectorClasses) {
+ if (isConcrete(connectorClass)) {
+ connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName()));
}
-
- validConnectorPlugins = connectorPlugins;
}
- {
- final Set> transformationClasses = reflections.getSubTypesOf(Transformation.class);
- transformationClasses.removeAll(TRANSFORMATION_EXCLUDES);
-
- final List> transformationPlugins = new LinkedList<>();
- for (Class extends Transformation> transformationClass : transformationClasses) {
- if (isConcrete(transformationClass)) {
- transformationPlugins.add(transformationClass);
- }
+ Collections.sort(connectorPlugins, new Comparator() {
+ @Override
+ public int compare(ConnectorPluginInfo a, ConnectorPluginInfo b) {
+ return a.clazz().compareTo(b.clazz());
}
+ });
- validTransformationPlugins = transformationPlugins;
- }
-
- scanned = true;
+ return connectorPlugins;
}
- public static synchronized List connectorPlugins() {
- scanClasspathForPlugins();
- return validConnectorPlugins;
- }
+ private static List> transformationPlugins(Reflections reflections) {
+ final Set> transformationClasses = reflections.getSubTypesOf(Transformation.class);
+ transformationClasses.removeAll(TRANSFORMATION_EXCLUDES);
- public static synchronized List> transformationPlugins() {
- scanClasspathForPlugins();
- return validTransformationPlugins;
+ final List> transformationPlugins = new LinkedList<>();
+ for (Class extends Transformation> transformationClass : transformationClasses) {
+ if (isConcrete(transformationClass)) {
+ transformationPlugins.add(transformationClass);
+ }
+ }
+
+ Collections.sort(transformationPlugins, new Comparator>() {
+ @Override
+ public int compare(Class extends Transformation> a, Class extends Transformation> b) {
+ return a.getCanonicalName().compareTo(b.getCanonicalName());
+ }
+ });
+
+ return transformationPlugins;
}
private static boolean isConcrete(Class> cls) {
From 06d14dba3c42d6184091bc5245407df6153b8bc1 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Mon, 9 Jan 2017 14:28:56 -0800
Subject: [PATCH 22/36] ArrayList over LinkedList
---
.../org/apache/kafka/connect/runtime/PluginDiscovery.java | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
index ccea6660c40d..847f527c43dd 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
@@ -32,10 +32,10 @@
import org.reflections.util.ConfigurationBuilder;
import java.lang.reflect.Modifier;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
-import java.util.LinkedList;
import java.util.List;
import java.util.Set;
@@ -76,7 +76,7 @@ private static List connectorPlugins(Reflections reflection
final Set> connectorClasses = reflections.getSubTypesOf(Connector.class);
connectorClasses.removeAll(CONNECTOR_EXCLUDES);
- final List connectorPlugins = new LinkedList<>();
+ final List connectorPlugins = new ArrayList<>(connectorClasses.size());
for (Class extends Connector> connectorClass : connectorClasses) {
if (isConcrete(connectorClass)) {
connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName()));
@@ -97,7 +97,7 @@ private static List> transformationPlugins(Refle
final Set> transformationClasses = reflections.getSubTypesOf(Transformation.class);
transformationClasses.removeAll(TRANSFORMATION_EXCLUDES);
- final List> transformationPlugins = new LinkedList<>();
+ final List> transformationPlugins = new ArrayList<>(transformationClasses.size());
for (Class extends Transformation> transformationClass : transformationClasses) {
if (isConcrete(transformationClass)) {
transformationPlugins.add(transformationClass);
From 52d5a13e2f526526a3863a87e470d63ed86d82d3 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 10 Jan 2017 10:44:06 -0800
Subject: [PATCH 23/36] Be better about type params, make TransformationChain
generic
---
.../connect/runtime/ConnectorConfig.java | 7 +++---
.../connect/runtime/TransformationChain.java | 22 ++++++++++---------
.../apache/kafka/connect/runtime/Worker.java | 14 +++++++-----
.../kafka/connect/runtime/WorkerSinkTask.java | 4 ++--
.../connect/runtime/WorkerSourceTask.java | 4 ++--
.../connect/runtime/ConnectorConfigTest.java | 6 ++---
.../connect/runtime/WorkerSinkTaskTest.java | 2 +-
.../runtime/WorkerSinkTaskThreadedTest.java | 2 +-
.../connect/runtime/WorkerSourceTaskTest.java | 2 +-
.../kafka/connect/runtime/WorkerTest.java | 6 ++---
10 files changed, 37 insertions(+), 32 deletions(-)
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
index 91790a3173c9..c779e0b17d7a 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
@@ -23,6 +23,7 @@
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigDef.Width;
import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.transforms.Transformation;
@@ -115,16 +116,16 @@ public ConnectorConfig(ConfigDef configDef, Map props) {
/**
* Returns the initialized list of {@link Transformation} which are specified in {@link #TRANSFORMS_CONFIG}.
*/
- public List transformations() {
+ public > List> transformations() {
final List transformAliases = getList(TRANSFORMS_CONFIG);
if (transformAliases == null || transformAliases.isEmpty()) {
return Collections.emptyList();
}
- final List transformations = new ArrayList<>(transformAliases.size());
+ final List> transformations = new ArrayList<>(transformAliases.size());
for (String alias : transformAliases) {
final String prefix = TRANSFORMS_CONFIG + "." + alias + ".";
- final Transformation transformation;
+ final Transformation transformation;
try {
transformation = getClass(prefix + "type").asSubclass(Transformation.class).newInstance();
} catch (Exception e) {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java
index 206b84e6ef82..b5a7dc4e46ab 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java
@@ -24,22 +24,19 @@
import java.util.List;
import java.util.Objects;
-public class TransformationChain {
+public class TransformationChain> {
- public static final TransformationChain NO_OP = new TransformationChain(Collections.emptyList());
+ private final List> transformations;
- private final List transformations;
-
- public TransformationChain(List transformations) {
+ public TransformationChain(List> transformations) {
this.transformations = transformations;
}
- @SuppressWarnings("unchecked")
- public > R apply(R record) {
+ public R apply(R record) {
if (transformations.isEmpty()) return record;
- for (Transformation transformation : transformations) {
- record = (R) transformation.apply(record);
+ for (Transformation transformation : transformations) {
+ record = transformation.apply(record);
if (record == null) break;
}
@@ -47,7 +44,7 @@ record = (R) transformation.apply(record);
}
public void close() {
- for (Transformation transformation : transformations) {
+ for (Transformation transformation : transformations) {
transformation.close();
}
}
@@ -64,4 +61,9 @@ public boolean equals(Object o) {
public int hashCode() {
return Objects.hash(transformations);
}
+
+ public static > TransformationChain noOp() {
+ return new TransformationChain(Collections.>emptyList());
+ }
+
}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
index f969259d632a..3703ed958e59 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
@@ -25,7 +25,9 @@
import org.apache.kafka.connect.connector.ConnectorContext;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.source.SourceTask;
import org.apache.kafka.connect.storage.Converter;
import org.apache.kafka.connect.storage.OffsetBackingStore;
@@ -325,9 +327,7 @@ public boolean startTask(
else
valueConverter = defaultValueConverter;
- final TransformationChain transformationChain = new TransformationChain(connConfig.transformations());
-
- workerTask = buildWorkerTask(id, task, statusListener, initialState, keyConverter, valueConverter, transformationChain);
+ workerTask = buildWorkerTask(connConfig, id, task, statusListener, initialState, keyConverter, valueConverter);
workerTask.initialize(taskConfig);
} catch (Throwable t) {
log.error("Failed to start task {}", id, t);
@@ -346,15 +346,16 @@ public boolean startTask(
return true;
}
- private WorkerTask buildWorkerTask(ConnectorTaskId id,
+ private WorkerTask buildWorkerTask(ConnectorConfig connConfig,
+ ConnectorTaskId id,
Task task,
TaskStatus.Listener statusListener,
TargetState initialState,
Converter keyConverter,
- Converter valueConverter,
- TransformationChain transformationChain) {
+ Converter valueConverter) {
// Decide which type of worker task we need based on the type of task.
if (task instanceof SourceTask) {
+ TransformationChain transformationChain = new TransformationChain<>(connConfig.transformations());
OffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(),
internalKeyConverter, internalValueConverter);
OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetBackingStore, id.connector(),
@@ -363,6 +364,7 @@ private WorkerTask buildWorkerTask(ConnectorTaskId id,
return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter,
valueConverter, transformationChain, producer, offsetReader, offsetWriter, config, time);
} else if (task instanceof SinkTask) {
+ TransformationChain transformationChain = new TransformationChain<>(connConfig.transformations());
return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, keyConverter,
valueConverter, transformationChain, time);
} else {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
index b8532130e17f..5cc70d5c1fb3 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
@@ -62,7 +62,7 @@ class WorkerSinkTask extends WorkerTask {
private final Time time;
private final Converter keyConverter;
private final Converter valueConverter;
- private final TransformationChain transformationChain;
+ private final TransformationChain transformationChain;
private KafkaConsumer consumer;
private WorkerSinkTaskContext context;
private final List messageBatch;
@@ -83,7 +83,7 @@ public WorkerSinkTask(ConnectorTaskId id,
WorkerConfig workerConfig,
Converter keyConverter,
Converter valueConverter,
- TransformationChain transformationChain,
+ TransformationChain transformationChain,
Time time) {
super(id, statusListener, initialState);
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
index 1a4fda859adc..74c3792606fb 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
@@ -55,7 +55,7 @@ class WorkerSourceTask extends WorkerTask {
private final SourceTask task;
private final Converter keyConverter;
private final Converter valueConverter;
- private final TransformationChain transformationChain;
+ private final TransformationChain transformationChain;
private KafkaProducer producer;
private final OffsetStorageReader offsetReader;
private final OffsetStorageWriter offsetWriter;
@@ -81,7 +81,7 @@ public WorkerSourceTask(ConnectorTaskId id,
TargetState initialState,
Converter keyConverter,
Converter valueConverter,
- TransformationChain transformationChain,
+ TransformationChain transformationChain,
KafkaProducer producer,
OffsetStorageReader offsetReader,
OffsetStorageWriter offsetWriter,
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java
index bd4f512b0e9d..9c77b363fc1b 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java
@@ -32,7 +32,7 @@
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
-public class ConnectorConfigTest {
+public class ConnectorConfigTest> {
public static abstract class TestConnector extends Connector {
}
@@ -125,7 +125,7 @@ public void singleTransform() {
props.put("transforms.a.type", SimpleTransformation.class.getName());
props.put("transforms.a.magic.number", "42");
final ConnectorConfig config = new ConnectorConfig(props);
- final List transformations = config.transformations();
+ final List> transformations = config.transformations();
assertEquals(1, transformations.size());
final SimpleTransformation xform = (SimpleTransformation) transformations.get(0);
assertEquals(42, xform.magicNumber);
@@ -153,7 +153,7 @@ public void multipleTransforms() {
props.put("transforms.b.type", SimpleTransformation.class.getName());
props.put("transforms.b.magic.number", "84");
final ConnectorConfig config = new ConnectorConfig(props);
- final List transformations = config.transformations();
+ final List> transformations = config.transformations();
assertEquals(2, transformations.size());
assertEquals(42, ((SimpleTransformation) transformations.get(0)).magicNumber);
assertEquals(84, ((SimpleTransformation) transformations.get(1)).magicNumber);
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
index 9adfdb5bc35c..f93e3859d016 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
@@ -109,7 +109,7 @@ public class WorkerSinkTaskTest {
@Mock
private Converter valueConverter;
@Mock
- private TransformationChain transformationChain;
+ private TransformationChain transformationChain;
@Mock
private TaskStatus.Listener statusListener;
@Mock
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
index 36caafccaeb1..9739c99e10e9 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
@@ -129,7 +129,7 @@ public void setup() {
workerConfig = new StandaloneConfig(workerProps);
workerTask = PowerMock.createPartialMock(
WorkerSinkTask.class, new String[]{"createConsumer"},
- taskId, sinkTask, statusListener, initialState, workerConfig, keyConverter, valueConverter, TransformationChain.NO_OP, time);
+ taskId, sinkTask, statusListener, initialState, workerConfig, keyConverter, valueConverter, TransformationChain.noOp(), time);
recordsReturned = 0;
}
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
index 8a9ef79d373d..8c07ba157f4f 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
@@ -85,7 +85,7 @@ public class WorkerSourceTaskTest extends ThreadedTest {
@Mock private SourceTask sourceTask;
@Mock private Converter keyConverter;
@Mock private Converter valueConverter;
- @Mock private TransformationChain transformationChain;
+ @Mock private TransformationChain transformationChain;
@Mock private KafkaProducer producer;
@Mock private OffsetStorageReader offsetReader;
@Mock private OffsetStorageWriter offsetWriter;
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java
index 39fd18e7c90e..4e90b4501274 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java
@@ -373,7 +373,7 @@ public void testAddRemoveTask() throws Exception {
EasyMock.eq(TargetState.STARTED),
EasyMock.anyObject(JsonConverter.class),
EasyMock.anyObject(JsonConverter.class),
- EasyMock.eq(TransformationChain.NO_OP),
+ EasyMock.eq(TransformationChain.noOp()),
EasyMock.anyObject(KafkaProducer.class),
EasyMock.anyObject(OffsetStorageReader.class),
EasyMock.anyObject(OffsetStorageWriter.class),
@@ -447,7 +447,7 @@ public void testCleanupTasksOnStop() throws Exception {
EasyMock.eq(TargetState.STARTED),
EasyMock.anyObject(JsonConverter.class),
EasyMock.anyObject(JsonConverter.class),
- EasyMock.eq(TransformationChain.NO_OP),
+ EasyMock.eq(TransformationChain.noOp()),
EasyMock.anyObject(KafkaProducer.class),
EasyMock.anyObject(OffsetStorageReader.class),
EasyMock.anyObject(OffsetStorageWriter.class),
@@ -502,7 +502,7 @@ public void testConverterOverrides() throws Exception {
EasyMock.eq(TargetState.STARTED),
EasyMock.capture(keyConverter),
EasyMock.capture(valueConverter),
- EasyMock.eq(TransformationChain.NO_OP),
+ EasyMock.eq(TransformationChain.noOp()),
EasyMock.anyObject(KafkaProducer.class),
EasyMock.anyObject(OffsetStorageReader.class),
EasyMock.anyObject(OffsetStorageWriter.class),
From 29195d35148619f3140f24c01377c6eeeba9d496 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 10 Jan 2017 10:55:51 -0800
Subject: [PATCH 24/36] unused import
---
.../connect/runtime/rest/resources/ConnectorPluginsResource.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java
index 1b56bc58531c..0d208022c3cb 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java
@@ -17,7 +17,6 @@
package org.apache.kafka.connect.runtime.rest.resources;
-import org.apache.kafka.connect.runtime.AbstractHerder;
import org.apache.kafka.connect.runtime.ConnectorConfig;
import org.apache.kafka.connect.runtime.Herder;
import org.apache.kafka.connect.runtime.PluginDiscovery;
From 672ca3bd8111000a84cf30dc5bfa45021ee9e079 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 10 Jan 2017 11:32:55 -0800
Subject: [PATCH 25/36] unused imports
---
.../java/org/apache/kafka/connect/runtime/AbstractHerder.java | 3 ---
1 file changed, 3 deletions(-)
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
index 4bb68c078577..1896ca451773 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
@@ -38,16 +38,13 @@
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
-import java.lang.reflect.Modifier;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
/**
From 81f9d7a4d2230f3be852e4994dbec07311733a55 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 10 Jan 2017 11:33:13 -0800
Subject: [PATCH 26/36] unused import
---
.../java/org/apache/kafka/connect/runtime/AbstractHerder.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
index 1896ca451773..cbcf14c33195 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
@@ -27,7 +27,6 @@
import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos;
import org.apache.kafka.connect.runtime.rest.entities.ConfigKeyInfo;
import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo;
-import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
import org.apache.kafka.connect.runtime.rest.errors.BadRequestException;
import org.apache.kafka.connect.source.SourceConnector;
From 60198a385c180257752835a2712cfb17efd926e7 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 10 Jan 2017 11:38:59 -0800
Subject: [PATCH 27/36] checkstyle fixen
---
checkstyle/import-control.xml | 1 +
.../connect/transforms/HoistKeyToStruct.java | 34 +++++++++----------
.../connect/transforms/HoistToStruct.java | 34 +++++++++----------
.../transforms/HoistValueToStruct.java | 34 +++++++++----------
.../kafka/connect/transforms/Insert.java | 15 ++++----
.../connect/transforms/TimestampRouter.java | 7 ++--
.../transforms/util/SchemaUpdateCache.java | 34 +++++++++----------
.../connect/transforms/util/SimpleConfig.java | 34 +++++++++----------
.../transforms/HoistKeyToStructTest.java | 34 +++++++++----------
9 files changed, 107 insertions(+), 120 deletions(-)
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 100f4f706c19..20d80b54613c 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -257,6 +257,7 @@
+
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
index 4ca07412c39d..af6ea07a2c8b 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
@@ -1,21 +1,19 @@
-/*
- * 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.
- */
+/**
+ * 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;
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
index 51d70121ba18..6d8fc72e9860 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
@@ -1,21 +1,19 @@
-/*
- * 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.
- */
+/**
+ * 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;
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
index 924b3fb470c8..27022d62e0de 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
@@ -1,21 +1,19 @@
-/*
- * 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.
- */
+/**
+ * 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;
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
index cdbd940dc39f..10d02a046523 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
@@ -35,14 +35,13 @@
abstract class Insert> implements Transformation {
- public enum Keys {
- ;
- public static final String TOPIC_FIELD = "topic.field";
- public static final String PARTITION_FIELD = "partition.field";
- public static final String OFFSET_FIELD = "offset.field";
- public static final String TIMESTAMP_FIELD = "timestamp.field";
- public static final String STATIC_FIELD = "static.field";
- public static final String STATIC_VALUE = "static.value";
+ public interface Keys {
+ 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 '!' to make this a required field, or '?' to keep it optional (the default).";
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
index 62c638173b6b..8a9cd3a290a9 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
@@ -35,10 +35,9 @@
*/
public class TimestampRouter> implements Transformation {
- public enum Keys {
- ;
- public static final String TOPIC_FORMAT = "topic.format";
- public static final String TIMESTAMP_FORMAT = "timestamp.format";
+ public interface Keys {
+ String TOPIC_FORMAT = "topic.format";
+ String TIMESTAMP_FORMAT = "timestamp.format";
}
private static final ConfigDef CONFIG_DEF = new ConfigDef()
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUpdateCache.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUpdateCache.java
index 1652f91cec9e..d2fecad37ea3 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUpdateCache.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUpdateCache.java
@@ -1,21 +1,19 @@
-/*
- * 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.
- */
+/**
+ * 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;
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java
index 12e149174f21..6ed1e14cb74e 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java
@@ -1,21 +1,19 @@
-/*
- * 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.
- */
+/**
+ * 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;
diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistKeyToStructTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistKeyToStructTest.java
index 29aa0b41613b..d67c56fe5ae6 100644
--- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistKeyToStructTest.java
+++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistKeyToStructTest.java
@@ -1,21 +1,19 @@
-/*
- * 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.
- */
+/**
+ * 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;
From f259080c4328a5d41dbd01ea237288a403ce008f Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 10 Jan 2017 12:57:03 -0800
Subject: [PATCH 28/36] Fix system test, make timestamp assertion robust
---
.../main/scala/kafka/tools/ConsoleConsumer.scala | 2 ++
tests/kafkatest/services/console_consumer.py | 10 ++++++++--
.../tests/connect/connect_distributed_test.py | 14 +++++++++-----
3 files changed, 19 insertions(+), 7 deletions(-)
diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
index eea66f8d170c..9affd937c43b 100755
--- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
+++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
@@ -464,6 +464,8 @@ class LoggingMessageFormatter extends MessageFormatter {
private val defaultWriter: DefaultMessageFormatter = new DefaultMessageFormatter
val logger = Logger.getLogger(getClass().getName)
+ override def init(props: Properties): Unit = defaultWriter.init(props)
+
def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream): Unit = {
import consumerRecord._
defaultWriter.writeTo(consumerRecord, output)
diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py
index 6984fc970ce7..17ddb6be26aa 100644
--- a/tests/kafkatest/services/console_consumer.py
+++ b/tests/kafkatest/services/console_consumer.py
@@ -89,7 +89,7 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService)
def __init__(self, context, num_nodes, kafka, topic, group_id="test-consumer-group", new_consumer=True,
message_validator=None, from_beginning=True, consumer_timeout_ms=None, version=TRUNK,
client_id="console-consumer", print_key=False, jmx_object_names=None, jmx_attributes=None,
- enable_systest_events=False, stop_timeout_sec=15):
+ enable_systest_events=False, stop_timeout_sec=15, print_timestamp=False):
"""
Args:
context: standard context
@@ -103,11 +103,12 @@ def __init__(self, context, num_nodes, kafka, topic, group_id="test-consumer-gro
successively consumed messages exceeds this timeout. Setting this and
waiting for the consumer to stop is a pretty good way to consume all messages
in a topic.
- print_key if True, print each message's key in addition to its value
+ print_key if True, print each message's key as well
enable_systest_events if True, console consumer will print additional lifecycle-related information
only available in 0.10.0 and later.
stop_timeout_sec After stopping a node, wait up to stop_timeout_sec for the node to stop,
and the corresponding background thread to finish successfully.
+ print_timestamp if True, print each message's timestamp as well
"""
JmxMixin.__init__(self, num_nodes, jmx_object_names, jmx_attributes or [])
BackgroundThreadService.__init__(self, context, num_nodes)
@@ -136,6 +137,8 @@ def __init__(self, context, num_nodes, kafka, topic, group_id="test-consumer-gro
# Only available in 0.10.0 and up
assert version >= V_0_10_0_0
+ self.print_timestamp = print_timestamp
+
def prop_file(self, node):
"""Return a string which can be used to create a configuration file appropriate for the given node."""
# Process client configuration
@@ -192,6 +195,9 @@ def start_cmd(self, node):
if self.print_key:
cmd += " --property print.key=true"
+ if self.print_timestamp:
+ cmd += " --property print.timestamp=true"
+
# LoggingMessageFormatter was introduced after 0.9
if node.version > LATEST_0_9:
cmd += " --formatter kafka.tools.LoggingMessageFormatter"
diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py
index db9191a0cc76..26418cb96556 100644
--- a/tests/kafkatest/tests/connect/connect_distributed_test.py
+++ b/tests/kafkatest/tests/connect/connect_distributed_test.py
@@ -444,7 +444,6 @@ def test_transformations(self):
self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node))
self.cc.start()
- start_time_ms = int(time.time() * 1000)
ts_fieldname = 'the_timestamp'
NamedConnector = namedtuple('Connector', ['name'])
@@ -461,7 +460,7 @@ def test_transformations(self):
'transforms.hoistToStruct.type': 'org.apache.kafka.connect.transforms.HoistValueToStruct',
'transforms.hoistToStruct.field': 'content',
'transforms.insertTimestampField.type': 'org.apache.kafka.connect.transforms.InsertInValue',
- 'transforms.insertTimestampField.timestamp': ts_fieldname,
+ 'transforms.insertTimestampField.timestamp.field': ts_fieldname,
})
wait_until(lambda: self.connector_is_running(source_connector), timeout_sec=30, err_msg='Failed to see connector transition to the RUNNING state')
@@ -469,7 +468,7 @@ def test_transformations(self):
for node in self.cc.nodes:
node.account.ssh("echo -e -n " + repr(self.FIRST_INPUTS) + " >> " + self.INPUT_FILE)
- consumer = ConsoleConsumer(self.test_context, 1, self.kafka, self.TOPIC, consumer_timeout_ms=15000)
+ consumer = ConsoleConsumer(self.test_context, 1, self.kafka, self.TOPIC, consumer_timeout_ms=15000, print_timestamp=True)
consumer.run()
assert len(consumer.messages_consumed[1]) == len(self.FIRST_INPUT_LIST)
@@ -484,10 +483,15 @@ def test_transformations(self):
}
for msg in consumer.messages_consumed[1]:
- obj = json.loads(msg)
+ (ts_info, value) = msg.split('\t')
+
+ assert ts_info.startswith('CreateTime:')
+ ts = int(ts_info[len('CreateTime:'):])
+
+ obj = json.loads(value)
assert obj['schema'] == expected_schema
assert obj['payload']['content'] in self.FIRST_INPUT_LIST
- assert obj['payload'][ts_fieldname] > start_time_ms
+ assert obj['payload'][ts_fieldname] == ts
def _validate_file_output(self, input):
input_set = set(input)
From b8e6a3d6074ab5b3f132204d0928fcd875b31fe8 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 10 Jan 2017 14:11:48 -0800
Subject: [PATCH 29/36] Make sure sourceTask.commitRecord() is called even for
filtered records
---
.../java/org/apache/kafka/connect/source/SourceTask.java | 6 +-----
.../org/apache/kafka/connect/runtime/WorkerSourceTask.java | 5 ++++-
2 files changed, 5 insertions(+), 6 deletions(-)
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
index 11477ea30004..c08508553dcd 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
@@ -82,17 +82,13 @@ public void commit() throws InterruptedException {
/**
*
- * Commit an individual {@link SourceRecord} when the callback from the producer client is received.
+ * Commit an individual {@link SourceRecord} when the callback from the producer client is received, or if a record is filtered by a transformation.
*
*
* SourceTasks are not required to implement this functionality; Kafka Connect will record offsets
* automatically. This hook is provided for systems that also need to store offsets internally
* in their own system.
*
- *
- * Note that it is not necessary that all records get a corresponding callback,
- * as filtering is possible with a user-configured {@link org.apache.kafka.connect.transforms.Transformation}.
- *
*
* @param record {@link SourceRecord} that was successfully sent via the producer.
* @throws InterruptedException
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
index 74c3792606fb..092072da873e 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
@@ -188,7 +188,10 @@ private boolean sendRecords() {
for (final SourceRecord preTransformRecord : toSend) {
final SourceRecord record = transformationChain.apply(preTransformRecord);
- if (record == null) continue;
+ if (record == null) {
+ commitTaskRecord(preTransformRecord);
+ continue;
+ }
byte[] key = keyConverter.fromConnectData(record.topic(), record.keySchema(), record.key());
byte[] value = valueConverter.fromConnectData(record.topic(), record.valueSchema(), record.value());
From 210b6c1b116d38cbe379b75b97fba0c8777e6be9 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Tue, 10 Jan 2017 14:52:19 -0800
Subject: [PATCH 30/36] Kill pre-mature optimizations on the SchemaUpdateCache
for a reasonably sized Synchronized
---
.../connect/transforms/HoistToStruct.java | 11 ++--
.../kafka/connect/transforms/Insert.java | 12 ++--
.../transforms/util/SchemaUpdateCache.java | 56 -------------------
3 files changed, 13 insertions(+), 66 deletions(-)
delete mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUpdateCache.java
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
index 6d8fc72e9860..9abd5eacf34f 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
@@ -17,12 +17,14 @@
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.SchemaUpdateCache;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.Map;
@@ -35,7 +37,7 @@ abstract class HoistToStruct> implements Transformati
.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.");
- private final SchemaUpdateCache schemaUpdateCache = new SchemaUpdateCache();
+ private Cache schemaUpdateCache;
private String fieldName;
@@ -43,8 +45,7 @@ abstract class HoistToStruct> implements Transformati
public void configure(Map props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
fieldName = config.getString("field");
-
- schemaUpdateCache.init();
+ schemaUpdateCache = new SynchronizedCache<>(new LRUCache(16));
}
@Override
@@ -65,7 +66,7 @@ public R apply(R record) {
@Override
public void close() {
- schemaUpdateCache.close();
+ schemaUpdateCache = null;
}
@Override
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
index 10d02a046523..e3f87ca020a4 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
@@ -17,6 +17,9 @@
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;
@@ -26,7 +29,6 @@
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.sink.SinkRecord;
-import org.apache.kafka.connect.transforms.util.SchemaUpdateCache;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.Date;
@@ -83,8 +85,6 @@ public static InsertionSpec parse(String spec) {
}
}
- private final SchemaUpdateCache schemaUpdateCache = new SchemaUpdateCache();
-
private InsertionSpec topicField;
private InsertionSpec partitionField;
private InsertionSpec offsetField;
@@ -93,6 +93,8 @@ public static InsertionSpec parse(String spec) {
private String staticValue;
private boolean applicable;
+ private Cache schemaUpdateCache;
+
@Override
public void configure(Map props) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
@@ -104,7 +106,7 @@ public void configure(Map props) {
staticValue = config.getString(Keys.STATIC_VALUE);
applicable = topicField != null || partitionField != null || offsetField != null || timestampField != null;
- schemaUpdateCache.init();
+ schemaUpdateCache = new SynchronizedCache<>(new LRUCache(16));
}
@Override
@@ -231,7 +233,7 @@ private void insertFields(R record, Struct value) {
@Override
public void close() {
- schemaUpdateCache.close();
+ schemaUpdateCache = null;
}
@Override
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUpdateCache.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUpdateCache.java
deleted file mode 100644
index d2fecad37ea3..000000000000
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUpdateCache.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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;
-
-/**
- * Single-entry thread-local cache. It uses reference equality to avoid expensive comparisons.
- */
-public class SchemaUpdateCache {
-
- private static final class Entry {
- final Schema base;
- final Schema updated;
-
- private Entry(Schema base, Schema updated) {
- this.base = base;
- this.updated = updated;
- }
- }
-
- private ThreadLocal cache;
-
- public void init() {
- cache = new ThreadLocal<>();
- }
-
- public Schema get(Schema base) {
- final Entry entry = cache.get();
- return entry != null && entry.base == base ? entry.updated : null;
- }
-
- public void put(Schema base, Schema updated) {
- cache.set(new Entry(base, updated));
- }
-
- public void close() {
- cache = null;
- }
-
-}
From 28a9b21b71a4fda052cd07a66ae6dfb09fe69364 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Wed, 11 Jan 2017 10:46:56 -0800
Subject: [PATCH 31/36] groupPrefix -> group
---
.../org/apache/kafka/connect/runtime/ConnectorConfig.java | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
index c779e0b17d7a..7086da2fdbdf 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
@@ -153,7 +153,7 @@ public static ConfigDef enrich(ConfigDef baseConfigDef, Map prop
for (String alias : new LinkedHashSet<>(transformAliases)) {
final String prefix = TRANSFORMS_CONFIG + "." + alias + ".";
- final String groupPrefix = TRANSFORMS_GROUP + ": " + alias;
+ final String group = TRANSFORMS_GROUP + ": " + alias;
int orderInGroup = 0;
final String transformationTypeConfig = prefix + "type";
@@ -164,7 +164,7 @@ public void ensureValid(String name, Object value) {
}
};
newDef.define(transformationTypeConfig, Type.CLASS, ConfigDef.NO_DEFAULT_VALUE, typeValidator, Importance.HIGH,
- "Class for the '" + alias + "' transformation.", groupPrefix, orderInGroup++, Width.LONG, "Transformation type for " + alias,
+ "Class for the '" + alias + "' transformation.", group, orderInGroup++, Width.LONG, "Transformation type for " + alias,
Collections.emptyList(), new TransformationClassRecommender());
final ConfigDef transformationConfigDef;
@@ -180,7 +180,7 @@ public void ensureValid(String name, Object value) {
}
}
- newDef.embed(prefix, groupPrefix, orderInGroup, transformationConfigDef);
+ newDef.embed(prefix, group, orderInGroup, transformationConfigDef);
}
return newDef;
From a4ee772d6d16d0e79c8a353c0488997451d793aa Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Wed, 11 Jan 2017 10:49:31 -0800
Subject: [PATCH 32/36] rm outdated doc
---
.../java/org/apache/kafka/connect/transforms/InsertInKey.java | 1 -
.../java/org/apache/kafka/connect/transforms/InsertInValue.java | 1 -
2 files changed, 2 deletions(-)
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
index 7003ca7c6bb0..3411c132bc2c 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
@@ -23,7 +23,6 @@
/**
* This transformation allows inserting configured attributes of the record metadata as fields in the record key.
* It also allows adding a static data field.
- * The record key is required to be of type {@link Schema.Type#STRUCT}.
*/
public class InsertInKey> extends Insert {
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
index 9aae2b714136..8f777ff819ad 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
@@ -23,7 +23,6 @@
/**
* This transformation allows inserting configured attributes of the record metadata as fields in the record value.
* It also allows adding a static data field.
- * The record value is required to be of type {@link Schema.Type#STRUCT}.
*/
public class InsertInValue> extends Insert {
From 30604f34a05df80f64a66b4a147ade92779db401 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Wed, 11 Jan 2017 10:54:41 -0800
Subject: [PATCH 33/36] note that filtering possible in xform.apply() javadoc
---
.../apache/kafka/connect/transforms/Transformation.java | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java b/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
index 7be3aa905c27..b17119ffff31 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java
@@ -30,7 +30,12 @@
*/
public interface Transformation> extends Configurable, Closeable {
- /** Apply transformation to the {@code record} and return another record object (which may be {@code record} itself). Must be thread-safe. **/
+ /**
+ * Apply transformation to the {@code record} and return another record object (which may be {@code record} itself) or {@code null},
+ * corresponding to a map or filter operation respectively.
+ *
+ * The implementation must be thread-safe.
+ */
R apply(R record);
/** Configuration specification for this transformation. **/
From 170d13473bcebf70b1821c01d4049dbf0f2ca27b Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Wed, 11 Jan 2017 11:08:00 -0800
Subject: [PATCH 34/36] Add kafkaPartition arg to ConnectRecord.newRecord()
---
.../connect/connector/ConnectRecord.java | 2 +-
.../apache/kafka/connect/sink/SinkRecord.java | 4 +--
.../kafka/connect/source/SourceRecord.java | 4 +--
.../connect/transforms/HoistKeyToStruct.java | 2 +-
.../transforms/HoistValueToStruct.java | 2 +-
.../kafka/connect/transforms/InsertInKey.java | 2 +-
.../connect/transforms/InsertInValue.java | 2 +-
.../connect/transforms/SetSchemaMetadata.java | 33 +++++++++++++++++++
.../connect/transforms/TimestampRouter.java | 2 +-
9 files changed, 43 insertions(+), 10 deletions(-)
create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java
index 8b812bf3cd81..6236f7e93682 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java
@@ -80,7 +80,7 @@ public Long timestamp() {
}
/** Generate a new record of the same type as itself, with the specified parameter values. **/
- public abstract R newRecord(String topic, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp);
+ public abstract R newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp);
@Override
public String toString() {
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java
index ec88f139ea4d..ad1b2d55e154 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java
@@ -55,8 +55,8 @@ public TimestampType timestampType() {
}
@Override
- public SinkRecord newRecord(String topic, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp) {
- return new SinkRecord(topic, kafkaPartition(), keySchema, key, valueSchema, value, kafkaOffset(), timestamp, timestampType);
+ public SinkRecord newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp) {
+ return new SinkRecord(topic, kafkaPartition, keySchema, key, valueSchema, value, kafkaOffset(), timestamp, timestampType);
}
@Override
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java
index 4dbeeb529a5c..444979a41dfb 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java
@@ -86,8 +86,8 @@ public SourceRecord(Map sourcePartition, Map sourceOffset,
}
@Override
- public SourceRecord newRecord(String topic, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp) {
- return new SourceRecord(sourcePartition, sourceOffset, topic, kafkaPartition(), keySchema, key, valueSchema, value, timestamp);
+ public SourceRecord newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp) {
+ return new SourceRecord(sourcePartition, sourceOffset, topic, kafkaPartition, keySchema, key, valueSchema, value, timestamp);
}
@Override
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
index af6ea07a2c8b..07cb7ffddc5e 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
@@ -37,7 +37,7 @@ protected Object operatingValue(R record) {
@Override
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
- return record.newRecord(record.topic(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp());
+ return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp());
}
}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
index 27022d62e0de..1e0086aa49b7 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
@@ -37,7 +37,7 @@ protected Object operatingValue(R record) {
@Override
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
- return record.newRecord(record.topic(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp());
+ return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp());
}
}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
index 3411c132bc2c..6a6a6e367611 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
@@ -38,7 +38,7 @@ protected Object operatingValue(R record) {
@Override
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
- return record.newRecord(record.topic(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp());
+ return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp());
}
}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
index 8f777ff819ad..13d79d817130 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
@@ -38,7 +38,7 @@ protected Object operatingValue(R record) {
@Override
protected R newRecord(R record, Schema updatedSchema, Object updatedValue) {
- return record.newRecord(record.topic(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp());
+ return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp());
}
}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
new file mode 100644
index 000000000000..47813d152358
--- /dev/null
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
@@ -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;
+
+import org.apache.kafka.common.config.ConfigDef;
+
+public class SetSchemaMetadata {
+
+ public interface Keys {
+ String SCHEMA_NAME = "schema.name";
+ String SCHEMA_VERSION = "schema.version";
+ }
+
+ private static final ConfigDef CONFIG_DEF = new ConfigDef()
+ .define(Keys.SCHEMA_NAME, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "Schema name to set.")
+ .define(Keys.SCHEMA_VERSION, ConfigDef.Type.INT, null, ConfigDef.Importance.HIGH, "Schema version to set.");
+
+}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
index 8a9cd3a290a9..1dd5345a6091 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java
@@ -75,7 +75,7 @@ public R apply(R record) {
final String formattedTimestamp = timestampFormat.get().format(new Date(timestamp));
final String updatedTopic = topicFormat.replace("${topic}", record.topic()).replace("${timestamp}", formattedTimestamp);
return record.newRecord(
- updatedTopic,
+ updatedTopic, record.kafkaPartition(),
record.keySchema(), record.key(),
record.valueSchema(), record.value(),
record.timestamp()
From 26281565bbc89915466546e7a8f9f915f661414e Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Wed, 11 Jan 2017 11:16:07 -0800
Subject: [PATCH 35/36] fix connect rest systest
---
tests/kafkatest/tests/connect/connect_rest_test.py | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/tests/kafkatest/tests/connect/connect_rest_test.py b/tests/kafkatest/tests/connect/connect_rest_test.py
index 098790b4c538..999d773729e9 100644
--- a/tests/kafkatest/tests/connect/connect_rest_test.py
+++ b/tests/kafkatest/tests/connect/connect_rest_test.py
@@ -31,8 +31,8 @@ class ConnectRestApiTest(KafkaTest):
FILE_SOURCE_CONNECTOR = 'org.apache.kafka.connect.file.FileStreamSourceConnector'
FILE_SINK_CONNECTOR = 'org.apache.kafka.connect.file.FileStreamSinkConnector'
- FILE_SOURCE_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'topic', 'file'}
- FILE_SINK_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'topics', 'file'}
+ FILE_SOURCE_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'topic', 'file', 'transforms'}
+ FILE_SINK_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'topics', 'file', 'transforms'}
INPUT_FILE = "/mnt/connect.input"
INPUT_FILE2 = "/mnt/connect.input2"
From 70cff21068e27e316525961181d3df5fdfedab53 Mon Sep 17 00:00:00 2001
From: Shikhar Bhushan
Date: Thu, 12 Jan 2017 11:39:48 -0800
Subject: [PATCH 36/36] Encapsulate key/value variants for data transformations
in base class
---
.../connect/transforms/HoistKeyToStruct.java | 43 -----------------
.../connect/transforms/HoistToStruct.java | 46 +++++++++++++++++-
.../transforms/HoistValueToStruct.java | 43 -----------------
.../{Insert.java => InsertField.java} | 48 ++++++++++++++++++-
.../kafka/connect/transforms/InsertInKey.java | 44 -----------------
.../connect/transforms/InsertInValue.java | 44 -----------------
.../connect/transforms/SetSchemaMetadata.java | 33 -------------
...StructTest.java => HoistToStructTest.java} | 4 +-
...tInValueTest.java => InsertFieldTest.java} | 8 ++--
.../tests/connect/connect_distributed_test.py | 4 +-
10 files changed, 100 insertions(+), 217 deletions(-)
delete mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
delete mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
rename connect/transforms/src/main/java/org/apache/kafka/connect/transforms/{Insert.java => InsertField.java} (86%)
delete mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInKey.java
delete mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertInValue.java
delete mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
rename connect/transforms/src/test/java/org/apache/kafka/connect/transforms/{HoistKeyToStructTest.java => HoistToStructTest.java} (93%)
rename connect/transforms/src/test/java/org/apache/kafka/connect/transforms/{InsertInValueTest.java => InsertFieldTest.java} (95%)
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
deleted file mode 100644
index 07cb7ffddc5e..000000000000
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistKeyToStruct.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.connector.ConnectRecord;
-import org.apache.kafka.connect.data.Schema;
-
-/**
- * Wraps the record key in a {@link org.apache.kafka.connect.data.Struct} with specified field name.
- */
-public class HoistKeyToStruct> extends HoistToStruct {
-
- @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());
- }
-
-}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
index 9abd5eacf34f..c2726ca84378 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java
@@ -29,7 +29,7 @@
import java.util.Map;
-abstract class HoistToStruct> implements Transformation {
+public abstract class HoistToStruct> implements Transformation {
public static final String FIELD_CONFIG = "field";
@@ -80,4 +80,48 @@ 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.
+ */
+ public static class Key> extends HoistToStruct {
+
+ @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());
+ }
+
+ }
+
+ /**
+ * Wraps the record value in a {@link org.apache.kafka.connect.data.Struct} with specified field name.
+ */
+ public static class Value> extends HoistToStruct {
+
+ @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());
+ }
+
+ }
+
}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
deleted file mode 100644
index 1e0086aa49b7..000000000000
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistValueToStruct.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.connector.ConnectRecord;
-import org.apache.kafka.connect.data.Schema;
-
-/**
- * Wraps the record value in a {@link org.apache.kafka.connect.data.Struct} with specified field name.
- */
-public class HoistValueToStruct> extends HoistToStruct {
-
- @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());
- }
-
-}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java
similarity index 86%
rename from connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
rename to connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java
index e3f87ca020a4..d67fea089323 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Insert.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java
@@ -35,7 +35,7 @@
import java.util.HashMap;
import java.util.Map;
-abstract class Insert> implements Transformation {
+public abstract class InsertField> implements Transformation {
public interface Keys {
String TOPIC_FIELD = "topic.field";
@@ -247,4 +247,50 @@ public ConfigDef config() {
protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue);
+ /**
+ * This transformation allows inserting configured attributes of the record metadata as fields in the record key.
+ * It also allows adding a static data field.
+ */
+ public static class Key