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



##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PredicatedTransformation.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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 java.util.Map;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.connect.connector.ConnectRecord;
+import org.apache.kafka.connect.transforms.Transformation;
+import org.apache.kafka.connect.transforms.predicates.Predicate;
+
+/**
+ * Decorator for a {@link Transformation} which applies the delegate only when 
a
+ * {@link Predicate} is true (or false, according to {@code negate}).
+ * @param <R>
+ */
+class PredicatedTransformation<R extends ConnectRecord<R>> implements 
Transformation<R> {
+
+    /*test*/ final Predicate<R> predicate;
+    /*test*/ final Transformation<R> delegate;
+    /*test*/ final boolean negate;
+
+    PredicatedTransformation(Predicate<R> predicate, boolean negate, 
Transformation<R> delegate) {
+        this.predicate = predicate;
+        this.negate = negate;
+        this.delegate = delegate;
+    }
+
+    @Override
+    public void configure(Map<String, ?> configs) {
+
+    }
+
+    @Override
+    public R apply(R record) {
+        if (negate ^ predicate.test(record)) {
+            return delegate.apply(record);
+        }
+        return record;
+    }
+
+    @Override
+    public ConfigDef config() {
+        return null;

Review comment:
       Hmmm... I think there might be some awkwardness here with trying to make 
`PredicatedTransformer` implement the `Transformation` interface. Could we 
replace every `Transformation` in the `TransformationChain`'s [transformation 
list](https://github.com/apache/kafka/blob/de6468ae5915298279e229dc64721e01e7d14fab/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java#L33)
 with a `PredicatedTransformer` and, if there are no predicates configured for 
a transform by the user, make the default behavior for the 
`PredicatedTransformer` class to blindly apply its transformation?
   
   This would solve a few problems:
   
   - No risk of users trying to actually use a `PredicatedTransformer` in a 
connector config, which they may try to do if we don't add logic to prevent it 
from being picked up during plugin path scanning on startup and logged as an 
SMT plugin
   - No need to implement methods that aren't used
   - One code path instead of two for application of transformations
   - More flexibility in instantiation and, possibly, the ability to 
encapsulate some of the `ConfigDef` generation logic in a separate class from 
`ConnectorConfig` (haven't looked into the specifics of this yet so may not 
actually be feasible or that elegant)

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
##########
@@ -257,12 +273,25 @@ public boolean includeRecordDetailsInErrorLog() {
         final List<Transformation<R>> transformations = new 
ArrayList<>(transformAliases.size());
         for (String alias : transformAliases) {
             final String prefix = TRANSFORMS_CONFIG + "." + alias + ".";
+
             try {
                 @SuppressWarnings("unchecked")
                 final Transformation<R> transformation = getClass(prefix + 
"type").asSubclass(Transformation.class)
                         .getDeclaredConstructor().newInstance();
-                transformation.configure(originalsWithPrefix(prefix));
-                transformations.add(transformation);
+                Map<String, Object> configs = originalsWithPrefix(prefix);
+                Object predicateAlias = configs.remove("predicate");
+                Object negate = configs.remove("negate");

Review comment:
       Hmmm... wish we'd caught that earlier. Seems safer to just leave the 
properties in, but unless we want to call for a re-vote and an extension on the 
KIP deadline guess we'll have to keep this as-is.




----------------------------------------------------------------
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