diff --git a/build.gradle b/build.gradle index 6f4d2501f7345..e191a51c3b89a 100644 --- a/build.gradle +++ b/build.gradle @@ -508,7 +508,7 @@ project(':core') { task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs', 'genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs', - 'genTopicConfigDocs', ':connect:runtime:genConnectConfigDocs', + 'genTopicConfigDocs', ':connect:runtime:genConnectConfigDocs', ':connect:runtime:genConnectTransformationDocs', ':streams:genStreamsConfigDocs'], type: Tar) { classifier = 'site-docs' compression = Compression.GZIP @@ -948,6 +948,13 @@ project(':connect:runtime') { if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } standardOutput = new File(generatedDocsDir, "connect_config.html").newOutputStream() } + + task genConnectTransformationDocs(type: JavaExec) { + classpath = sourceSets.main.runtimeClasspath + main = 'org.apache.kafka.connect.tools.TransformationDoc' + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "connect_transforms.html").newOutputStream() + } } project(':connect:file') { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java new file mode 100644 index 0000000000000..6746042baa743 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *
+ * 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.tools; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.transforms.ExtractField; +import org.apache.kafka.connect.transforms.HoistField; +import org.apache.kafka.connect.transforms.InsertField; +import org.apache.kafka.connect.transforms.MaskField; +import org.apache.kafka.connect.transforms.RegexRouter; +import org.apache.kafka.connect.transforms.ReplaceField; +import org.apache.kafka.connect.transforms.SetSchemaMetadata; +import org.apache.kafka.connect.transforms.TimestampRouter; +import org.apache.kafka.connect.transforms.ValueToKey; + +import java.io.PrintStream; +import java.util.Arrays; +import java.util.List; + +public class TransformationDoc { + + private static final class DocInfo { + final String transformationName; + final String overview; + final ConfigDef configDef; + + private DocInfo(String transformationName, String overview, ConfigDef configDef) { + this.transformationName = transformationName; + this.overview = overview; + this.configDef = configDef; + } + } + + private static final List" + Key.class.getCanonicalName() + ") "
+ + "or value (" + Value.class.getCanonicalName() + ").";
+
+ private static final String FIELD_CONFIG = "field";
+
+ public static final ConfigDef CONFIG_DEF = new ConfigDef()
+ .define(FIELD_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM, "Field name to extract.");
+
+ private static final String PURPOSE = "field extraction";
+
+ private String fieldName;
+
+ @Override
+ public void configure(Map" + Key.class.getCanonicalName() + ") "
+ + "or value (" + Value.class.getCanonicalName() + ").";
- private static final ConfigDef CONFIG_DEF = new ConfigDef()
+ private static final String FIELD_CONFIG = "field";
+
+ public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(FIELD_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM,
- "Field name for the single field that will be created in the resulting Struct.");
+ "Field name for the single field that will be created in the resulting Struct or Map.");
private Cache" + Key.class.getCanonicalName() + ") "
+ + "or value (" + Value.class.getCanonicalName() + ").";
+
+ private interface ConfigName {
String TOPIC_FIELD = "topic.field";
String PARTITION_FIELD = "partition.field";
String OFFSET_FIELD = "offset.field";
@@ -46,22 +55,24 @@ public interface Keys {
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_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
- "Field name for Kafka topic.\n" + OPTIONALITY_DOC)
- .define(Keys.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
- "Field name for Kafka partition.\n" + OPTIONALITY_DOC)
- .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_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,
+ private static final String OPTIONALITY_DOC = "Suffix with ! to make this a required field, or ? to keep it optional (the default).";
+
+ public static final ConfigDef CONFIG_DEF = new ConfigDef()
+ .define(ConfigName.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ "Field name for Kafka topic. " + OPTIONALITY_DOC)
+ .define(ConfigName.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ "Field name for Kafka partition. " + OPTIONALITY_DOC)
+ .define(ConfigName.OFFSET_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+ "Field name for Kafka offset - only applicable to sink connectors." + Key.class.getCanonicalName() + ") "
+ + "or value (" + Value.class.getCanonicalName() + ").";
+
+ private static final String FIELDS_CONFIG = "fields";
+
+ public static final ConfigDef CONFIG_DEF = new ConfigDef()
+ .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Names of fields to mask.");
+
+ private static final String PURPOSE = "mask fields";
+
+ private static final Map+ * http://www.apache.org/licenses/LICENSE-2.0 + *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+
+package org.apache.kafka.connect.transforms;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.ConnectRecord;
+import org.apache.kafka.connect.transforms.util.RegexValidator;
+import org.apache.kafka.connect.transforms.util.SimpleConfig;
+
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class RegexRouter
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+
+package org.apache.kafka.connect.transforms;
+
+import org.apache.kafka.common.cache.Cache;
+import org.apache.kafka.common.cache.LRUCache;
+import org.apache.kafka.common.cache.SynchronizedCache;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.connector.ConnectRecord;
+import org.apache.kafka.connect.data.Field;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.SchemaBuilder;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.transforms.util.SchemaUtil;
+import org.apache.kafka.connect.transforms.util.SimpleConfig;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.kafka.connect.transforms.util.Requirements.requireMap;
+import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct;
+
+public abstract class ReplaceField
+ * 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.common.config.ConfigException;
+import org.apache.kafka.connect.connector.ConnectRecord;
+import org.apache.kafka.connect.data.ConnectSchema;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.transforms.util.SimpleConfig;
+
+import java.util.Map;
+
+import static org.apache.kafka.connect.transforms.util.Requirements.requireSchema;
+
+public abstract class SetSchemaMetadata
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+
+package org.apache.kafka.connect.transforms;
+
+import org.apache.kafka.common.cache.Cache;
+import org.apache.kafka.common.cache.LRUCache;
+import org.apache.kafka.common.cache.SynchronizedCache;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.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.NonEmptyListValidator;
+import org.apache.kafka.connect.transforms.util.SimpleConfig;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.kafka.connect.transforms.util.Requirements.requireMap;
+import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct;
+
+public class ValueToKey
+ * 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.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+
+import java.util.List;
+
+public class NonEmptyListValidator implements ConfigDef.Validator {
+
+ @Override
+ public void ensureValid(String name, Object value) {
+ if (((List) value).isEmpty()) {
+ throw new ConfigException(name, value, "Empty list");
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "non-empty list";
+ }
+
+}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java
new file mode 100644
index 0000000000000..9713b27446f47
--- /dev/null
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java
@@ -0,0 +1,41 @@
+/**
+ * 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.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+
+import java.util.regex.Pattern;
+
+public class RegexValidator implements ConfigDef.Validator {
+
+ @Override
+ public void ensureValid(String name, Object value) {
+ try {
+ Pattern.compile((String) value);
+ } catch (Exception e) {
+ throw new ConfigException(name, value, "Invalid regex: " + e.getMessage());
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "valid regex";
+ }
+
+}
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java
new file mode 100644
index 0000000000000..b004f8ae91500
--- /dev/null
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java
@@ -0,0 +1,61 @@
+/**
+ * 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.connector.ConnectRecord;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.errors.DataException;
+import org.apache.kafka.connect.sink.SinkRecord;
+
+import java.util.Map;
+
+public class Requirements {
+
+ public static void requireSchema(Schema schema, String purpose) {
+ if (schema == null) {
+ throw new DataException("Schema required for [" + purpose + "]");
+ }
+ }
+
+ public static Map
+ * 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.sink.SinkRecord;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class RegexRouterTest {
+
+ private static String apply(String regex, String replacement, String topic) {
+ final Map
+ * 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.data.Schema;
+import org.apache.kafka.connect.data.SchemaBuilder;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class ReplaceFieldTest {
+
+ @Test
+ public void schemaless() {
+ final ReplaceField
+ * 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.data.SchemaBuilder;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class SetSchemaMetadataTest {
+
+ @Test
+ public void schemaNameUpdate() {
+ final SetSchemaMetadata
+ * 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.data.Schema;
+import org.apache.kafka.connect.data.SchemaBuilder;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class ValueToKeyTest {
+
+ @Test
+ public void schemaless() {
+ final ValueToKeyjava.util.regex.Pattern. "
+ + "If the pattern matches the input topic, java.util.regex.Matcher#replaceFirst() is used with the replacement string to obtain the new topic.";
+
+ public static final ConfigDef CONFIG_DEF = new ConfigDef()
+ .define(ConfigName.REGEX, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, new RegexValidator(), ConfigDef.Importance.HIGH,
+ "Regular expression to use for matching.")
+ .define(ConfigName.REPLACEMENT, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.HIGH,
+ "Replacement string.");
+
+ private interface ConfigName {
+ String REGEX = "regex";
+ String REPLACEMENT = "replacement";
+ }
+
+ private Pattern regex;
+ private String replacement;
+
+ @Override
+ public void configure(Mapfoo:bar,abc:xyz";
+ }
+ }, ConfigDef.Importance.MEDIUM, "Field rename mappings.");
+
+ private static final String PURPOSE = "field replacement";
+
+ private List" + Key.class.getCanonicalName() + ")"
+ + " or value (" + Value.class.getCanonicalName() + ") schema.";
+
+ private interface ConfigName {
+ String SCHEMA_NAME = "schema.name";
+ String SCHEMA_VERSION = "schema.version";
+ }
+
+ public static final ConfigDef CONFIG_DEF = new ConfigDef()
+ .define(ConfigName.SCHEMA_NAME, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "Schema name to set.")
+ .define(ConfigName.SCHEMA_VERSION, ConfigDef.Type.INT, null, ConfigDef.Importance.HIGH, "Schema version to set.");
+
+ private String schemaName;
+ private Integer schemaVersion;
+
+ @Override
+ public void configure(Map${topic} and ${timestamp} as placeholders for the topic and timestamp, respectively.")
+ .define(ConfigName.TIMESTAMP_FORMAT, ConfigDef.Type.STRING, "yyyyMMdd", ConfigDef.Importance.HIGH,
+ "Format string for the timestamp that is compatible with java.text.SimpleDateFormat.");
+
+ private interface ConfigName {
String TOPIC_FORMAT = "topic.format";
String TIMESTAMP_FORMAT = "timestamp.format";
}
- private static final ConfigDef CONFIG_DEF = new ConfigDef()
- .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(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 ThreadLocalConfiguring Connecto
For any other options, you should consult the documentation for the connector.
+
Transformations
+
+ Connectors can be configured with transformations to make lightweight message-at-a-time modifications. They can be convenient for minor data massaging and routing changes.
+
+ A transformation chain can be specified in the connector configuration.
+
+
+
+
+ Several widely-applicable data and routing transformations are included with Kafka Connect:
+
+
+
transforms - List of aliases for the transformation, specifying the order in which the transformations will be applied.transforms.$alias.type - Fully qualified class name for the transformation.transforms.$alias.$transformationSpecificConfig Configuration properties for the transformationREST API
Since Kafka Connect is intended to be run as a service, it also provides a REST API for managing connectors. By default, this service runs on port 8083. The following are the currently supported endpoints:
diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py
index 198e94501f50d..c298fb154e86e 100644
--- a/tests/kafkatest/tests/connect/connect_distributed_test.py
+++ b/tests/kafkatest/tests/connect/connect_distributed_test.py
@@ -460,7 +460,7 @@ def test_transformations(self):
'file': self.INPUT_FILE,
'topic': self.TOPIC,
'transforms': 'hoistToStruct,insertTimestampField',
- 'transforms.hoistToStruct.type': 'org.apache.kafka.connect.transforms.HoistToStruct$Value',
+ 'transforms.hoistToStruct.type': 'org.apache.kafka.connect.transforms.HoistField$Value',
'transforms.hoistToStruct.field': 'content',
'transforms.insertTimestampField.type': 'org.apache.kafka.connect.transforms.InsertField$Value',
'transforms.insertTimestampField.timestamp.field': ts_fieldname,