C0urante commented on a change in pull request #9549:
URL: https://github.com/apache/kafka/pull/9549#discussion_r523147909



##########
File path: 
connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.common.config.ConfigException;
+import org.apache.kafka.connect.connector.ConnectRecord;
+import org.apache.kafka.connect.data.Field;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.SchemaBuilder;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.header.Headers;
+import org.apache.kafka.connect.transforms.util.Requirements;
+import org.apache.kafka.connect.transforms.util.SchemaUtil;
+import org.apache.kafka.connect.transforms.util.SimpleConfig;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE;
+
+public abstract class HeaderFrom<R extends ConnectRecord<R>> implements 
Transformation<R> {
+
+    public static final String FIELDS_FIELD = "fields";
+    public static final String HEADERS_FIELD = "headers";
+    public static final String OPERATION_FIELD = "operation";
+
+    public static final String OVERVIEW_DOC =
+            "Moves or copies fields in the key/value of a record into that 
record's headers. " +
+                    "Corresponding elements of <code>" + FIELDS_FIELD + 
"</code> and " +
+                    "<code>" + HEADERS_FIELD + "</code> together identify a 
field and the header it should be " +
+                    "moved or copied to. " +
+                    "Use the concrete transformation type designed for the 
record " +
+                    "key (<code>" + Key.class.getName() + "</code>) or value 
(<code>" + Value.class.getName() + "</code>).";
+
+    public static final ConfigDef CONFIG_DEF = new ConfigDef()
+            .define(FIELDS_FIELD, ConfigDef.Type.LIST, 
ConfigDef.Importance.HIGH,
+                    "Field names in the record whose values are to be copied 
or moved to headers.")
+            .define(HEADERS_FIELD, ConfigDef.Type.LIST, 
ConfigDef.Importance.HIGH,
+                    "Header names, in the same order as the field names listed 
in the fields configuration property.")
+            .define(OPERATION_FIELD, ConfigDef.Type.STRING, NO_DEFAULT_VALUE,
+                    ConfigDef.ValidString.in("move", "copy"), 
ConfigDef.Importance.HIGH,
+                    "Either <code>move</code> if the fields are to be moved to 
the headers (removed from the key/value), " +
+                            "or <code>copy</code> if the fields are to be 
copied to the headers (retained in the key/value).");
+
+    enum Operation {
+        MOVE("move"),
+        COPY("copy");
+
+        private final String name;
+
+        Operation(String name) {
+            this.name = name;
+        }
+
+        static Operation fromName(String name) {
+            switch (name) {
+                case "move":
+                    return MOVE;
+                case "copy":
+                    return COPY;
+                default:
+                    throw new IllegalArgumentException();
+            }
+        }
+
+        public String toString() {
+            return name;
+        }
+    }
+
+    private List<String> fields;
+
+    private List<String> headers;
+
+    private Operation operation;
+
+    @Override
+    public R apply(R record) {
+        Object operatingValue = operatingValue(record);
+        Schema operatingSchema = operatingSchema(record);
+
+        if (operatingSchema == null) {
+            return applySchemaless(record, operatingValue);
+        } else {
+            return applyWithSchema(record, operatingValue, operatingSchema);
+        }
+    }
+
+    private R applyWithSchema(R record, Object operatingValue, Schema 
operatingSchema) {
+        Headers updatedHeaders = record.headers().duplicate();

Review comment:
       Ah, that's fair. Does seem to be the pattern followed by the other 
out-of-the-box transformations as well; probably best to continue to follow 
that pattern.
   
   I'm a little unnerved by this though, since as far as I can tell it's not 
publicly documented and so it's possible people writing their own 
transformations may be violating this implicit rule.
   
   Out of scope, so I've filed 
[KAFKA-10720](https://issues.apache.org/jira/browse/KAFKA-10720) to track the 
need for possible documentation improvements.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to