EndzeitBegins commented on code in PR #8049:
URL: https://github.com/apache/nifi/pull/8049#discussion_r1404891409


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttributes.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.nifi.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.documentation.UseCase;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "delete", "Attribute 
Expression Language"})
+@CapabilityDescription("Filters the Attributes for a FlowFile against a set of 
attribute names to retain or remove")
+@UseCase(
+        description = "Retain only a specified set of FlowFile attributes",
+        configuration = """
+        Define a "Delimiter" that does not occur in the names of any of the 
attributes to retain.
+        Specify the set of "Attributes to filter" using the delimiter defined 
before.
+        Set "Filter mode" to "Retain".
+        """
+)
+public class FilterAttributes extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this 
relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = 
Collections.singleton(REL_SUCCESS);
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new 
PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. 
Each attribute name is separated by the delimiter.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor DELIMITER = new 
PropertyDescriptor.Builder()
+            .name("DELIMITER")
+            .displayName("Delimiter")
+            .description("One or multiple characters that separates one 
attribute name value from another.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(",")
+            .build();
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new 
AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other 
attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new 
AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes 
are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new 
PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered 
attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, 
FILTER_MODE_VALUE_REMOVE)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    private final static List<PropertyDescriptor> properties = 
List.of(ATTRIBUTE_SET, DELIMITER, FILTER_MODE);
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    private Set<String> preCalculatedAttributes;
+
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        if (context.getProperty(ATTRIBUTE_SET).isExpressionLanguagePresent()
+                || 
context.getProperty(DELIMITER).isExpressionLanguagePresent()) {
+            // the attribute set may rely on FlowFile attributes; thus we 
cannot pre-calculate them
+            preCalculatedAttributes = null;
+        } else {
+            preCalculatedAttributes = calculateAttributeSet(context, null);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession 
session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final Set<String> declaredAttributes;
+        if (preCalculatedAttributes != null) {
+            declaredAttributes = preCalculatedAttributes;
+        } else {
+            declaredAttributes = calculateAttributeSet(context, flowFile);
+        }
+
+        final FilterMode filterMode = getFilterMode(context, flowFile);
+        final Set<String> attributesToRemove = 
computeAttributesToRemove(flowFile, filterMode, declaredAttributes);
+
+        session.removeAllAttributes(flowFile, attributesToRemove);
+        session.transfer(flowFile, REL_SUCCESS);
+    }
+
+    private static Set<String> calculateAttributeSet(ProcessContext context, 
FlowFile flowFile) {
+        final String attributeSet = getAttributeSet(context, flowFile);
+        final String delimiter = getDelimiter(context, flowFile);
+
+        return parseAttributeSet(attributeSet, delimiter);
+    }
+
+    private static String getAttributeSet(ProcessContext context, FlowFile 
flowFile) {
+        PropertyValue attributeSetProperty = 
context.getProperty(ATTRIBUTE_SET);
+
+        if (flowFile != null) {

Review Comment:
   While we no longer pre-calculate / cache the attributeSet but a 
`Predicate<String>` instead, this is necessary as we can calculate it 
`@OnScheduled`, when no NiFi EL is used, or per FlowFile in the `onTrigger` 
whenever NiFi EL is used.
   In the former case, the `flowFile` attribute is nullable.
   
   Is this okay to you, or should this be refactored to make this more visible, 
e.g. using `Optional` or method overloads with and without `FlowFile flowFile` 
parameter?



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttributes.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.nifi.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.documentation.UseCase;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "delete", "Attribute 
Expression Language"})
+@CapabilityDescription("Filters the Attributes for a FlowFile against a set of 
attribute names to retain or remove")
+@UseCase(
+        description = "Retain only a specified set of FlowFile attributes",
+        configuration = """
+        Define a "Delimiter" that does not occur in the names of any of the 
attributes to retain.
+        Specify the set of "Attributes to filter" using the delimiter defined 
before.
+        Set "Filter mode" to "Retain".
+        """
+)
+public class FilterAttributes extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this 
relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = 
Collections.singleton(REL_SUCCESS);
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new 
PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. 
Each attribute name is separated by the delimiter.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor DELIMITER = new 
PropertyDescriptor.Builder()
+            .name("DELIMITER")
+            .displayName("Delimiter")
+            .description("One or multiple characters that separates one 
attribute name value from another.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(",")
+            .build();
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new 
AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other 
attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new 
AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes 
are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new 
PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered 
attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, 
FILTER_MODE_VALUE_REMOVE)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    private final static List<PropertyDescriptor> properties = 
List.of(ATTRIBUTE_SET, DELIMITER, FILTER_MODE);
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    private Set<String> preCalculatedAttributes;

Review Comment:
   Changed.



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

To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org

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

Reply via email to